[ASTERIXDB-3119][*DB][IDX] Add query-index() function to scan secondary indexes
- user model changes: no
- storage format changes: no
- interface changes: no
Change-Id: Id149c26c75bb7fe2902f91ccd0c20824106b36a3
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17397
Reviewed-by: Ali Alsuliman <ali.al.solaiman@gmail.com>
Reviewed-by: Wail Alkowaileet <wael.y.k@gmail.com>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index ecf0352..25d42df 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -31,6 +31,7 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
+import java.util.stream.Collectors;
import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -434,7 +435,10 @@
dest.addAll(KeyFieldTypeUtil.getArrayBTreeIndexKeyTypes(index, recordType, metaRecordType));
break;
case BTREE:
- dest.addAll(KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
+ //TODO(ali): check if types should be made nullable/missable
+ List<Pair<IAType, Boolean>> bTreeIndexKeyTypes =
+ KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType);
+ dest.addAll(bTreeIndexKeyTypes.stream().map(Pair::getFirst).collect(Collectors.toList()));
break;
case RTREE:
dest.addAll(KeyFieldTypeUtil.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index dfe29c2..0b2975f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -80,6 +80,12 @@
throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
"No positional variables are allowed over datasource functions");
}
+ createDataScanOp(opRef, unnest, context, f);
+ return true;
+ }
+
+ protected void createDataScanOp(Mutable<ILogicalOperator> opRef, UnnestOperator unnest,
+ IOptimizationContext context, AbstractFunctionCallExpression f) throws AlgebricksException {
FunctionDataSource datasource = toDatasource(context, f);
List<LogicalVariable> variables = new ArrayList<>();
variables.add(unnest.getVariable());
@@ -89,7 +95,6 @@
scanInpList.addAll(unnest.getInputs());
opRef.setValue(scan);
context.computeAndSetTypeEnvironmentForOperator(scan);
- return true;
}
protected abstract FunctionDataSource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
new file mode 100644
index 0000000..f43588e
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.function;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+
+public class QueryIndexDatasource extends FunctionDataSource {
+
+ private final Dataset ds;
+ private final String indexName;
+ private final AlgebricksAbsolutePartitionConstraint storageLocations;
+
+ public QueryIndexDatasource(Dataset ds, String indexName, INodeDomain domain,
+ AlgebricksAbsolutePartitionConstraint storageLocations, ARecordType recType) throws AlgebricksException {
+ super(createQueryIndexDataSourceId(ds, indexName), QueryIndexRewriter.QUERY_INDEX, domain, recType);
+ this.ds = ds;
+ this.indexName = indexName;
+ this.storageLocations = storageLocations;
+ }
+
+ @Override
+ protected void initSchemaType(IAType iType) {
+ ARecordType type = (ARecordType) iType;
+ IAType[] fieldTypes = type.getFieldTypes();
+ schemaTypes = new IAType[fieldTypes.length];
+ for (int i = 0; i < schemaTypes.length; i++) {
+ schemaTypes[i] = fieldTypes[i];
+ }
+ }
+
+ @Override
+ protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
+ return storageLocations;
+ }
+
+ @Override
+ public boolean isScanAccessPathALeaf() {
+ // the index scan op is not a leaf op. the ETS op will start the scan of the index. we need the ETS op below
+ // the index scan to be still generated
+ return false;
+ }
+
+ @Override
+ protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+ AlgebricksAbsolutePartitionConstraint locations) {
+ throw new UnsupportedOperationException("query-index() does not use record reader adapter");
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+ MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+ IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+ IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, ds, indexName,
+ null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false);
+ }
+
+ @Override
+ public IDataSourcePropertiesProvider getPropertiesProvider() {
+ return scanVariables -> {
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<>(1);
+ int numScanKeys = scanVariables.size();
+ List<OrderColumn> scanKeys = new ArrayList<>(numScanKeys);
+ for (int i = 0; i < numScanKeys; i++) {
+ scanKeys.add(new OrderColumn(scanVariables.get(i), OrderOperator.IOrder.OrderKind.ASC));
+ }
+ propsLocal.add(new LocalOrderProperty(scanKeys));
+ return new StructuralPropertiesVector(new RandomPartitioningProperty(domain), propsLocal);
+ };
+ }
+
+ private static DataSourceId createQueryIndexDataSourceId(Dataset dataset, String indexName) {
+ return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(),
+ new String[] { indexName, QueryIndexRewriter.QUERY_INDEX.getName() });
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java
new file mode 100644
index 0000000..1906fcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexRewriter.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.function;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
+import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+public class QueryIndexRewriter extends FunctionRewriter implements IResultTypeComputer {
+
+ public static final FunctionIdentifier QUERY_INDEX =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "query-index", 3);
+ public static final QueryIndexRewriter INSTANCE = new QueryIndexRewriter(QUERY_INDEX);
+
+ private QueryIndexRewriter(FunctionIdentifier functionId) {
+ super(functionId);
+ }
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env, IMetadataProvider<?, ?> mp)
+ throws AlgebricksException {
+ return computeRecType((AbstractFunctionCallExpression) expression, (MetadataProvider) mp, null, null, null);
+ }
+
+ @Override
+ public FunctionDataSource toDatasource(IOptimizationContext ctx, AbstractFunctionCallExpression f)
+ throws AlgebricksException {
+ final SourceLocation loc = f.getSourceLocation();
+ DataverseName dvName = getDataverseName(loc, f.getArguments(), 0);
+ String dsName = getString(loc, f.getArguments(), 1);
+ String idName = getString(loc, f.getArguments(), 2);
+ MetadataProvider mp = (MetadataProvider) ctx.getMetadataProvider();
+ final Dataset dataset = validateDataset(mp, dvName, dsName, loc);
+ Index index = validateIndex(f, mp, loc, dvName, dsName, idName);
+ return createQueryIndexDatasource(mp, dataset, index, loc, f);
+ }
+
+ @Override
+ protected void createDataScanOp(Mutable<ILogicalOperator> opRef, UnnestOperator unnest, IOptimizationContext ctx,
+ AbstractFunctionCallExpression f) throws AlgebricksException {
+ FunctionDataSource datasource = toDatasource(ctx, f);
+ List<LogicalVariable> variables = new ArrayList<>();
+ List<Mutable<ILogicalExpression>> closedRecArgs = new ArrayList<>();
+ MetadataProvider mp = (MetadataProvider) ctx.getMetadataProvider();
+ computeRecType(f, mp, variables, closedRecArgs, ctx);
+ DataSourceScanOperator scan = new DataSourceScanOperator(variables, datasource);
+ scan.setSourceLocation(unnest.getSourceLocation());
+ List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
+ scanInpList.addAll(unnest.getInputs());
+ ScalarFunctionCallExpression recordCreationFunc = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR), closedRecArgs);
+ recordCreationFunc.setSourceLocation(unnest.getSourceLocation());
+ AssignOperator assignOp = new AssignOperator(unnest.getVariable(), new MutableObject<>(recordCreationFunc));
+ assignOp.getInputs().add(new MutableObject<>(scan));
+ assignOp.setSourceLocation(unnest.getSourceLocation());
+ ctx.computeAndSetTypeEnvironmentForOperator(scan);
+ ctx.computeAndSetTypeEnvironmentForOperator(assignOp);
+ opRef.setValue(assignOp);
+ }
+
+ private FunctionDataSource createQueryIndexDatasource(MetadataProvider mp, Dataset ds, Index idx,
+ SourceLocation loc, AbstractFunctionCallExpression f) throws AlgebricksException {
+ ISecondaryIndexOperationsHelper secIdxHelper =
+ SecondaryIndexOperationsHelper.createIndexOperationsHelper(ds, idx, mp, loc);
+ new IndexDataflowHelperFactory(mp.getStorageComponentProvider().getStorageManager(),
+ secIdxHelper.getSecondaryFileSplitProvider());
+ AlgebricksAbsolutePartitionConstraint secPartitionConstraint =
+ (AlgebricksAbsolutePartitionConstraint) secIdxHelper.getSecondaryPartitionConstraint();
+ INodeDomain domain = mp.findNodeDomain(ds.getNodeGroupName());
+ ARecordType recType = computeRecType(f, mp, null, null, null);
+ return new QueryIndexDatasource(ds, idx.getIndexName(), domain, secPartitionConstraint, recType);
+ }
+
+ private ARecordType computeRecType(AbstractFunctionCallExpression f, MetadataProvider metadataProvider,
+ List<LogicalVariable> outVars, List<Mutable<ILogicalExpression>> closedRecArgs,
+ IOptimizationContext context) throws AlgebricksException {
+ final SourceLocation loc = f.getSourceLocation();
+ DataverseName dataverseName = getDataverseName(loc, f.getArguments(), 0);
+ String datasetName = getString(loc, f.getArguments(), 1);
+ String indexName = getString(loc, f.getArguments(), 2);
+ Dataset dataset = validateDataset(metadataProvider, dataverseName, datasetName, loc);
+ Index index = validateIndex(f, metadataProvider, loc, dataverseName, datasetName, indexName);
+ ARecordType dsType = (ARecordType) metadataProvider.findType(dataset);
+ ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+ List<IAType> dsKeyTypes = KeyFieldTypeUtil.getPartitoningKeyTypes(dataset, dsType, metaType);
+ List<Pair<IAType, Boolean>> secKeyTypes = KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, dsType, metaType);
+ int numPrimaryKeys = dsKeyTypes.size();
+ int numSecKeys = secKeyTypes.size();
+ String[] fieldNames = new String[numSecKeys + numPrimaryKeys];
+ IAType[] fieldTypes = new IAType[numSecKeys + numPrimaryKeys];
+ int keyIdx = 0;
+ boolean overridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
+ for (int i = 0; i < numSecKeys; i++, keyIdx++) {
+ IAType secKeyType = secKeyTypes.get(i).first;
+ Boolean makeOptional = secKeyTypes.get(i).second;
+ fieldTypes[keyIdx] =
+ overridingKeyFieldTypes || makeOptional ? AUnionType.createUnknownableType(secKeyType) : secKeyType;
+ fieldNames[keyIdx] = "SK" + i;
+ setAssignVarsExprs(outVars, closedRecArgs, context, loc, fieldNames, keyIdx);
+ }
+ for (int k = 0; k < numPrimaryKeys; k++, keyIdx++) {
+ fieldTypes[keyIdx] = dsKeyTypes.get(k);
+ fieldNames[keyIdx] = "PK" + k;
+ setAssignVarsExprs(outVars, closedRecArgs, context, loc, fieldNames, keyIdx);
+ }
+ return new ARecordType("", fieldNames, fieldTypes, false);
+ }
+
+ private void setAssignVarsExprs(List<LogicalVariable> outVars, List<Mutable<ILogicalExpression>> closedRecArgs,
+ IOptimizationContext context, SourceLocation loc, String[] fieldNames, int n) {
+ if (context != null) {
+ LogicalVariable logicalVariable = context.newVar();
+ outVars.add(logicalVariable);
+ ConstantExpression nameExpr = new ConstantExpression(new AsterixConstantValue(new AString(fieldNames[n])));
+ VariableReferenceExpression varRefExpr = new VariableReferenceExpression(logicalVariable);
+ nameExpr.setSourceLocation(loc);
+ varRefExpr.setSourceLocation(loc);
+ closedRecArgs.add(new MutableObject<>(nameExpr));
+ closedRecArgs.add(new MutableObject<>(varRefExpr));
+ }
+ }
+
+ private static Dataset validateDataset(MetadataProvider mp, DataverseName dvName, String dsName, SourceLocation loc)
+ throws AlgebricksException {
+ Dataset dataset = mp.findDataset(dvName, dsName);
+ if (dataset == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, loc, dsName, dvName);
+ }
+ return dataset;
+ }
+
+ private static Index validateIndex(AbstractFunctionCallExpression f, MetadataProvider mp, SourceLocation loc,
+ DataverseName dvName, String dsName, String idxName) throws AlgebricksException {
+ Index index = mp.getIndex(dvName, dsName, idxName);
+ if (index == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_INDEX, loc, idxName);
+ }
+ if (index.isPrimaryIndex()) {
+ throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, idxName);
+ }
+ DatasetConfig.IndexType idxType = index.getIndexType();
+ // currently, only normal secondary indexes are supported
+ if (idxType != DatasetConfig.IndexType.BTREE || Index.IndexCategory.of(idxType) != Index.IndexCategory.VALUE
+ || index.isPrimaryKeyIndex()) {
+ throw new CompilationException(ErrorCode.COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX,
+ f.getSourceLocation(), LogRedactionUtil.userData(f.toString()));
+ }
+ return index;
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index d6dc67c..5a2ef3c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -26,6 +26,7 @@
import org.apache.asterix.app.function.FeedRewriter;
import org.apache.asterix.app.function.JobSummariesRewriter;
import org.apache.asterix.app.function.PingRewriter;
+import org.apache.asterix.app.function.QueryIndexRewriter;
import org.apache.asterix.app.function.StorageComponentsRewriter;
import org.apache.asterix.app.function.TPCDSAllTablesDataGeneratorRewriter;
import org.apache.asterix.app.function.TPCDSSingleTableDataGeneratorRewriter;
@@ -95,6 +96,10 @@
(expression, env, mp) -> RecordUtil.FULLY_OPEN_RECORD_TYPE, true);
BuiltinFunctions.addUnnestFun(DumpIndexRewriter.DUMP_INDEX, false);
BuiltinFunctions.addDatasourceFunction(DumpIndexRewriter.DUMP_INDEX, DumpIndexRewriter.INSTANCE);
+ // Query index function
+ BuiltinFunctions.addFunction(QueryIndexRewriter.QUERY_INDEX, QueryIndexRewriter.INSTANCE, true);
+ BuiltinFunctions.addUnnestFun(QueryIndexRewriter.QUERY_INDEX, false);
+ BuiltinFunctions.addDatasourceFunction(QueryIndexRewriter.QUERY_INDEX, QueryIndexRewriter.INSTANCE);
}
private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.sqlpp
new file mode 100644
index 0000000..31934ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.001.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int };
+
+CREATE DATASET ds1(t1) PRIMARY KEY id;
+
+CREATE INDEX ds1_array_idx ON ds1(UNNEST a : string) EXCLUDE UNKNOWN KEY;
+CREATE PRIMARY INDEX pk_idx ON ds1;
+ANALYZE DATASET ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp
new file mode 100644
index 0000000..796f598
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+// cannot use primary index
+FROM query_index("test", "ds1", "ds1") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp
new file mode 100644
index 0000000..f5f5774
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.003.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+// cannot use primary key index
+FROM query_index("test", "ds1", "pk_idx") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp
new file mode 100644
index 0000000..9bea68e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.004.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+// cannot use samples index
+FROM query_index("test", "ds1", "sample_idx_1_ds1") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp
new file mode 100644
index 0000000..cf3a0e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.005.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+// cannot use array index
+FROM query_index("test", "ds1", "ds1_array_idx") as v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/negative/negative.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp
new file mode 100644
index 0000000..270a1d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.001.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int };
+CREATE TYPE t2 AS { id: int, age: int };
+CREATE TYPE t3 AS { id: int, age: int?, dept: string? };
+
+CREATE DATASET ds1(t1) PRIMARY KEY id;
+CREATE DATASET ds2(t2) PRIMARY KEY id;
+CREATE DATASET ds3(t3) PRIMARY KEY id;
+
+CREATE INDEX ds1_age ON ds1(age: int);
+CREATE INDEX ds1_dept ON ds1(dept: string);
+CREATE INDEX ds1_age_dept ON ds1(age: int, dept: string);
+CREATE INDEX ds1_dept_age ON ds1(dept: string, age: int);
+
+CREATE INDEX ds2_age ON ds2(age);
+CREATE INDEX ds2_dept ON ds2(dept: string);
+CREATE INDEX ds2_age_dept ON ds2(age, dept: string);
+CREATE INDEX ds2_dept_age ON ds2(dept: string, age);
+
+CREATE INDEX ds3_age ON ds3(age);
+CREATE INDEX ds3_dept ON ds3(dept);
+CREATE INDEX ds3_age_dept ON ds3(age, dept);
+CREATE INDEX ds3_dept_age ON ds3(dept, age);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp
new file mode 100644
index 0000000..04fd66f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.002.update.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+UPSERT INTO ds1 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":3, "age": null, "dept": "cs"},
+{"id":4 , "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"},
+{"id":11 },
+{"id":12, "age": null, "dept": null},
+{"id":13, "age": null },
+{"id":14, "dept": null}
+]);
+
+UPSERT INTO ds2 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"}
+]);
+
+UPSERT INTO ds3 ([
+{"id":1, "age": 34, "dept": "cs"},
+{"id":2, "age": 20, "dept": "ms"},
+{"id":3, "age": null, "dept": "cs"},
+{"id":4 , "dept": "ms"},
+{"id":5, "age": 34, "dept": null},
+{"id":6, "age": 34 },
+{"id":7, "age": 20, "dept": "cs"},
+{"id":8, "age": 34, "dept": "ms"},
+{"id":9, "age": 34, "dept": "cs"},
+{"id":10, "age": 34, "dept": "ms"},
+{"id":11 },
+{"id":12, "age": null, "dept": null},
+{"id":13, "age": null },
+{"id":14, "dept": null}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp
new file mode 100644
index 0000000..1910ffc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.003.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds1", "ds1_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp
new file mode 100644
index 0000000..f801201
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.004.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds1", "ds1_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp
new file mode 100644
index 0000000..729a4f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.005.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds1", "ds1_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp
new file mode 100644
index 0000000..2914aab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.006.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds1", "ds1_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp
new file mode 100644
index 0000000..2ed609d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.007.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds3", "ds3_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp
new file mode 100644
index 0000000..504cae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.008.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds3", "ds3_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp
new file mode 100644
index 0000000..2272bbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.009.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds3", "ds3_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp
new file mode 100644
index 0000000..9ddc171
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.010.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds3", "ds3_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp
new file mode 100644
index 0000000..41f3505
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.011.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds2", "ds2_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp
new file mode 100644
index 0000000..bce624f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.012.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds2", "ds2_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp
new file mode 100644
index 0000000..62c5622
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.013.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds2", "ds2_age_dept") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp
new file mode 100644
index 0000000..f4f0560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.014.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM query_index("test", "ds2", "ds2_dept_age") as v
+SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp
new file mode 100644
index 0000000..31d5fbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.015.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.query.sqlpp
new file mode 100644
index 0000000..5e7caad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.016.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+FROM query_index("test", "ds1", "ds1_dept") as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp
new file mode 100644
index 0000000..12e1311
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.017.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.age, COUNT(*) AS cnt FROM ds1 as v GROUP BY v.age ORDER BY v.age;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp
new file mode 100644
index 0000000..7015ff7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.018.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp
new file mode 100644
index 0000000..dc94475
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.019.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp
new file mode 100644
index 0000000..4b2379d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.020.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.SK1 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_dept_age") as v GROUP BY v.SK1 ORDER BY v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp
new file mode 100644
index 0000000..ebb2269d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.021.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.age, v.dept, COUNT(*) AS cnt FROM ds1 as v GROUP BY v.age, v.dept ORDER BY v.age, v.dept;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp
new file mode 100644
index 0000000..b535ee8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.022.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+GROUP BY v.SK0, v.SK1
+ORDER BY v.SK0, v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp
new file mode 100644
index 0000000..87d3aad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.023.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+GROUP BY v.SK1, v.SK0
+ORDER BY v.SK1, v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp
new file mode 100644
index 0000000..8f6255f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.024.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN FROM query_index("test", "ds1", "ds1_age") as v SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp
new file mode 100644
index 0000000..55f2b70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.025.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN FROM query_index("test", "ds1", "ds1_age_dept") as v SELECT VALUE v ORDER BY v.PK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp
new file mode 100644
index 0000000..94b5f60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.026.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+ GROUP BY v.SK1, v.SK0
+ ORDER BY v.SK1, v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp
new file mode 100644
index 0000000..7cc70b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.027.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+EXPLAIN FROM query_index("test", "ds1", "ds1_dept") as v SELECT COUNT(*) AS cnt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp
new file mode 100644
index 0000000..e6008a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.028.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp
new file mode 100644
index 0000000..361c57f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.029.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v GROUP BY v.SK0 ORDER BY v.SK0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp
new file mode 100644
index 0000000..bcae31e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.030.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK1 AS age, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_dept_age") as v GROUP BY v.SK1 ORDER BY v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp
new file mode 100644
index 0000000..4b2dc99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.031.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+SET `compiler.sort.parallel` "false";
+EXPLAIN SELECT v.SK0 AS age, v.SK1 AS dept, COUNT(*) AS cnt FROM query_index("test", "ds1", "ds1_age_dept") as v
+ GROUP BY v.SK0, v.SK1
+ ORDER BY v.SK0, v.SK1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/query_index/q01/q01.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm
new file mode 100644
index 0000000..1b32b5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.003.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": null, "PK0": 3 }
+{ "PK0": 4 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm
new file mode 100644
index 0000000..7d4626e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.004.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": "cs", "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm
new file mode 100644
index 0000000..e5d1135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.005.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": null, "SK1": "cs", "PK0": 3 }
+{ "SK1": "ms", "PK0": 4 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "SK1": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm
new file mode 100644
index 0000000..d0d8cc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.006.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": "cs", "SK1": null, "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK1": null, "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm
new file mode 100644
index 0000000..1b32b5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.007.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": null, "PK0": 3 }
+{ "PK0": 4 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm
new file mode 100644
index 0000000..7d4626e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.008.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": "cs", "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "PK0": 12 }
+{ "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm
new file mode 100644
index 0000000..e5d1135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.009.adm
@@ -0,0 +1,14 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": null, "SK1": "cs", "PK0": 3 }
+{ "SK1": "ms", "PK0": 4 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK0": null, "PK0": 13 }
+{ "SK1": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm
new file mode 100644
index 0000000..d0d8cc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.010.adm
@@ -0,0 +1,14 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": "cs", "SK1": null, "PK0": 3 }
+{ "SK0": "ms", "PK0": 4 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
+{ "PK0": 11 }
+{ "SK0": null, "SK1": null, "PK0": 12 }
+{ "SK1": null, "PK0": 13 }
+{ "SK0": null, "PK0": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm
new file mode 100644
index 0000000..2a57ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.011.adm
@@ -0,0 +1,8 @@
+{ "SK0": 34, "PK0": 1 }
+{ "SK0": 20, "PK0": 2 }
+{ "SK0": 34, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "PK0": 7 }
+{ "SK0": 34, "PK0": 8 }
+{ "SK0": 34, "PK0": 9 }
+{ "SK0": 34, "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm
new file mode 100644
index 0000000..0021c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.012.adm
@@ -0,0 +1,8 @@
+{ "SK0": "cs", "PK0": 1 }
+{ "SK0": "ms", "PK0": 2 }
+{ "SK0": null, "PK0": 5 }
+{ "PK0": 6 }
+{ "SK0": "cs", "PK0": 7 }
+{ "SK0": "ms", "PK0": 8 }
+{ "SK0": "cs", "PK0": 9 }
+{ "SK0": "ms", "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm
new file mode 100644
index 0000000..1858896
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.013.adm
@@ -0,0 +1,8 @@
+{ "SK0": 34, "SK1": "cs", "PK0": 1 }
+{ "SK0": 20, "SK1": "ms", "PK0": 2 }
+{ "SK0": 34, "SK1": null, "PK0": 5 }
+{ "SK0": 34, "PK0": 6 }
+{ "SK0": 20, "SK1": "cs", "PK0": 7 }
+{ "SK0": 34, "SK1": "ms", "PK0": 8 }
+{ "SK0": 34, "SK1": "cs", "PK0": 9 }
+{ "SK0": 34, "SK1": "ms", "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm
new file mode 100644
index 0000000..3ce6c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.014.adm
@@ -0,0 +1,8 @@
+{ "SK0": "cs", "SK1": 34, "PK0": 1 }
+{ "SK0": "ms", "SK1": 20, "PK0": 2 }
+{ "SK0": null, "SK1": 34, "PK0": 5 }
+{ "SK1": 34, "PK0": 6 }
+{ "SK0": "cs", "SK1": 20, "PK0": 7 }
+{ "SK0": "ms", "SK1": 34, "PK0": 8 }
+{ "SK0": "cs", "SK1": 34, "PK0": 9 }
+{ "SK0": "ms", "SK1": 34, "PK0": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm
new file mode 100644
index 0000000..dfec480
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.015.adm
@@ -0,0 +1 @@
+{ "cnt": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm
new file mode 100644
index 0000000..dfec480
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.016.adm
@@ -0,0 +1 @@
+{ "cnt": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm
new file mode 100644
index 0000000..831b0cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.017.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "cnt": 3, "age": null }
+{ "cnt": 2, "age": 20 }
+{ "cnt": 6, "age": 34 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.018.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.019.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm
new file mode 100644
index 0000000..6a94e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.020.adm
@@ -0,0 +1,4 @@
+{ "cnt": 3 }
+{ "age": null, "cnt": 3 }
+{ "age": 20, "cnt": 2 }
+{ "age": 34, "cnt": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm
new file mode 100644
index 0000000..3c82bb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.021.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "cnt": 1, "dept": null }
+{ "cnt": 1, "dept": "ms" }
+{ "cnt": 1, "age": null }
+{ "cnt": 1, "age": null, "dept": null }
+{ "cnt": 1, "age": null, "dept": "cs" }
+{ "cnt": 1, "age": 20, "dept": "cs" }
+{ "cnt": 1, "age": 20, "dept": "ms" }
+{ "cnt": 1, "age": 34 }
+{ "cnt": 1, "age": 34, "dept": null }
+{ "cnt": 2, "age": 34, "dept": "cs" }
+{ "cnt": 2, "age": 34, "dept": "ms" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm
new file mode 100644
index 0000000..8dd327f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.022.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "dept": null, "cnt": 1 }
+{ "dept": "ms", "cnt": 1 }
+{ "age": null, "cnt": 1 }
+{ "age": null, "dept": null, "cnt": 1 }
+{ "age": null, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "ms", "cnt": 1 }
+{ "age": 34, "cnt": 1 }
+{ "age": 34, "dept": null, "cnt": 1 }
+{ "age": 34, "dept": "cs", "cnt": 2 }
+{ "age": 34, "dept": "ms", "cnt": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm
new file mode 100644
index 0000000..7cf5585
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.023.adm
@@ -0,0 +1,12 @@
+{ "cnt": 1 }
+{ "age": null, "cnt": 1 }
+{ "age": 34, "cnt": 1 }
+{ "dept": null, "cnt": 1 }
+{ "age": null, "dept": null, "cnt": 1 }
+{ "age": 34, "dept": null, "cnt": 1 }
+{ "age": null, "dept": "cs", "cnt": 1 }
+{ "age": 20, "dept": "cs", "cnt": 1 }
+{ "age": 34, "dept": "cs", "cnt": 2 }
+{ "dept": "ms", "cnt": 1 }
+{ "age": 20, "dept": "ms", "cnt": 1 }
+{ "age": 34, "dept": "ms", "cnt": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
new file mode 100644
index 0000000..45c7028
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.024.plan
@@ -0,0 +1,20 @@
+distribute result [$$v] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "PK0": $$14}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ order (ASC, $$14) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14] <- test.ds1.ds1_age.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
new file mode 100644
index 0000000..0308f76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.025.plan
@@ -0,0 +1,20 @@
+distribute result [$$v] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$v]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$v] <- [{"SK0": $$13, "SK1": $$14, "PK0": $$15}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
+ order (ASC, $$15) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$13, $$14, $$15] <- test.ds1.ds1_age_dept.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
new file mode 100644
index 0000000..2648cf3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.026.plan
@@ -0,0 +1,38 @@
+distribute result [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$SK1(ASC), $$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK1 := $$61; $$SK0 := $$62]) decor ([]) {
+ aggregate [$$59] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$61, $$62] |PARTITIONED|
+ group by ([$$61 := $$57; $$62 := $$56]) decor ([]) {
+ aggregate [$$60] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$57, $$56] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$57, $$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$57, $$58] <- test.ds1.ds1_age_dept.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
new file mode 100644
index 0000000..58e4a62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.027.plan
@@ -0,0 +1,22 @@
+distribute result [$$31] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$31]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$31] <- [{"cnt": $$34}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$34] <- [agg-sql-sum($$35)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$35] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$32, $$33] <- test.ds1.ds1_dept.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
new file mode 100644
index 0000000..c86f9fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.028.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$46}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$48]) decor ([]) {
+ aggregate [$$46] <- [agg-sql-sum($$47)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$48] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
+ group by ([$$48 := $$44]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45] <- test.ds1.ds1_age.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
new file mode 100644
index 0000000..e537bf3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.029.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK0, "cnt": $$47}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$49]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ group by ([$$49 := $$44]) decor ([]) {
+ aggregate [$$48] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$44] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$44]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45, $$46] <- test.ds1.ds1_age_dept.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
new file mode 100644
index 0000000..24aa758
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.030.plan
@@ -0,0 +1,38 @@
+distribute result [$$42] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$42]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$42] <- [{"age": $$SK1, "cnt": $$47}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$SK1(ASC) ] |PARTITIONED|
+ group by ([$$SK1 := $$49]) decor ([]) {
+ aggregate [$$47] <- [agg-sql-sum($$48)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$49] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ group by ([$$49 := $$45]) decor ([]) {
+ aggregate [$$48] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$45] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$44, $$45, $$46] <- test.ds1.ds1_dept_age.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
new file mode 100644
index 0000000..f2046b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/query_index/q01/q01.031.plan
@@ -0,0 +1,38 @@
+distribute result [$$53] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$53] <- [{"age": $$SK0, "dept": $$SK1, "cnt": $$59}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$SK0(ASC), $$SK1(ASC) ] |PARTITIONED|
+ group by ([$$SK0 := $$61; $$SK1 := $$62]) decor ([]) {
+ aggregate [$$59] <- [agg-sql-sum($$60)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_GROUP_BY[$$61, $$62] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$61, $$62] |PARTITIONED|
+ group by ([$$61 := $$56; $$62 := $$57]) decor ([]) {
+ aggregate [$$60] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- PRE_CLUSTERED_GROUP_BY[$$56, $$57] |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$56, $$57]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$56, $$57, $$58] <- test.ds1.ds1_age_dept.query-index [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 304520b..17df689 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9720,6 +9720,22 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="query_index">
+ <test-case FilePath="query_index">
+ <compilation-unit name="q01">
+ <output-dir compare="Text">q01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="query_index">
+ <compilation-unit name="negative">
+ <output-dir compare="Text">negative</output-dir>
+ <expected-error>ASX1105: Operation not supported on primary index ds1</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "pk_idx") cannot utilize index</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "sample_idx_1_ds1") cannot utilize index</expected-error>
+ <expected-error>ASX1026: The given function expression query-index("test", "ds1", "ds1_array_idx") cannot utilize index</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="range-hints">
<test-case FilePath="range-hints">
<compilation-unit name="order-by">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index f5fd7dd..a7d560c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -53,10 +53,19 @@
protected final FunctionIdentifier functionId;
+ public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain, IAType itemType)
+ throws AlgebricksException {
+ super(id, itemType, null, DataSource.Type.FUNCTION, domain);
+ this.functionId = functionId;
+ initSchemaType(itemType);
+ }
+
public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain)
throws AlgebricksException {
- super(id, RecordUtil.FULLY_OPEN_RECORD_TYPE, null, DataSource.Type.FUNCTION, domain);
- this.functionId = functionId;
+ this(id, functionId, domain, RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ }
+
+ protected void initSchemaType(IAType itemType) {
schemaTypes = new IAType[] { itemType };
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index c4cc1de..fd20b3c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -128,16 +128,16 @@
* @return a list of IATypes, one for each corresponding index key field.
* @throws AlgebricksException
*/
- public static List<IAType> getBTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
- throws AlgebricksException {
+ public static List<Pair<IAType, Boolean>> getBTreeIndexKeyTypes(Index index, ARecordType recordType,
+ ARecordType metaRecordType) throws AlgebricksException {
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
- List<IAType> indexKeyTypes = new ArrayList<>();
+ List<Pair<IAType, Boolean>> indexKeyTypes = new ArrayList<>();
for (int i = 0; i < indexDetails.getKeyFieldNames().size(); i++) {
Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i),
chooseSource(keySourceIndicators, i, recordType, metaRecordType));
- indexKeyTypes.add(keyPairType.first);
+ indexKeyTypes.add(keyPairType);
}
return indexKeyTypes;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
index 8b4e56d..5b75cd9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSource.java
@@ -36,6 +36,12 @@
public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList);
// https://issues.apache.org/jira/browse/ASTERIXDB-1619
+
+ /**
+ * Return true if this data source is the start of the job pipeline making its scan op the start of the job pipeline
+ * instead of an ETS op, for example. This flag is used to disable the Hyracks op generation of the input
+ * operators to the data scan (i.e. ETS op that is an input to the data scan will not be generated).
+ */
public boolean isScanAccessPathALeaf();
public INodeDomain getDomain();