You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2023/02/27 20:29:13 UTC

[asterixdb] 13/16: [ASTERIXDB-3119][*DB][IDX] Add query-index() function to scan secondary indexes

This is an automated email from the ASF dual-hosted git repository.

alsuliman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit dda25305ba3f9d62b00cb2468222414c6e4d4609
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Fri Feb 24 16:18:24 2023 -0800

    [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 <al...@gmail.com>
    Reviewed-by: Wail Alkowaileet <wa...@gmail.com>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
 .../optimizer/rules/am/AccessMethodUtils.java      |   6 +-
 .../asterix/app/function/FunctionRewriter.java     |   7 +-
 .../app/function/QueryIndexDatasource.java}        |  97 ++++++----
 .../asterix/app/function/QueryIndexRewriter.java   | 210 +++++++++++++++++++++
 .../asterix/util/MetadataBuiltinFunctions.java     |   5 +
 .../query_index/negative/negative.001.ddl.sqlpp    |  30 +++
 .../query_index/negative/negative.002.query.sqlpp  |  22 +++
 .../query_index/negative/negative.003.query.sqlpp  |  22 +++
 .../query_index/negative/negative.004.query.sqlpp  |  22 +++
 .../query_index/negative/negative.005.query.sqlpp  |  22 +++
 .../query_index/negative/negative.999.ddl.sqlpp    |  20 ++
 .../query_index/q01/q01.001.ddl.sqlpp              |  45 +++++
 .../query_index/q01/q01.002.update.sqlpp           |  65 +++++++
 .../query_index/q01/q01.003.query.sqlpp            |  23 +++
 .../query_index/q01/q01.004.query.sqlpp            |  23 +++
 .../query_index/q01/q01.005.query.sqlpp            |  23 +++
 .../query_index/q01/q01.006.query.sqlpp            |  23 +++
 .../query_index/q01/q01.007.query.sqlpp            |  23 +++
 .../query_index/q01/q01.008.query.sqlpp            |  23 +++
 .../query_index/q01/q01.009.query.sqlpp            |  23 +++
 .../query_index/q01/q01.010.query.sqlpp            |  23 +++
 .../query_index/q01/q01.011.query.sqlpp            |  23 +++
 .../query_index/q01/q01.012.query.sqlpp            |  23 +++
 .../query_index/q01/q01.013.query.sqlpp            |  23 +++
 .../query_index/q01/q01.014.query.sqlpp            |  23 +++
 .../query_index/q01/q01.015.query.sqlpp            |  22 +++
 .../query_index/q01/q01.016.query.sqlpp            |  21 +++
 .../query_index/q01/q01.017.query.sqlpp            |  22 +++
 .../query_index/q01/q01.018.query.sqlpp            |  22 +++
 .../query_index/q01/q01.019.query.sqlpp            |  22 +++
 .../query_index/q01/q01.020.query.sqlpp            |  22 +++
 .../query_index/q01/q01.021.query.sqlpp            |  22 +++
 .../query_index/q01/q01.022.query.sqlpp            |  24 +++
 .../query_index/q01/q01.023.query.sqlpp            |  24 +++
 .../query_index/q01/q01.024.query.sqlpp            |  22 +++
 .../query_index/q01/q01.025.query.sqlpp            |  22 +++
 .../query_index/q01/q01.026.query.sqlpp            |  24 +++
 .../query_index/q01/q01.027.query.sqlpp            |  22 +++
 .../query_index/q01/q01.028.query.sqlpp            |  22 +++
 .../query_index/q01/q01.029.query.sqlpp            |  22 +++
 .../query_index/q01/q01.030.query.sqlpp            |  22 +++
 .../query_index/q01/q01.031.query.sqlpp            |  24 +++
 .../query_index/q01/q01.999.ddl.sqlpp              |  20 ++
 .../runtimets/results/query_index/q01/q01.003.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.004.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.005.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.006.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.007.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.008.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.009.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.010.adm  |  14 ++
 .../runtimets/results/query_index/q01/q01.011.adm  |   8 +
 .../runtimets/results/query_index/q01/q01.012.adm  |   8 +
 .../runtimets/results/query_index/q01/q01.013.adm  |   8 +
 .../runtimets/results/query_index/q01/q01.014.adm  |   8 +
 .../runtimets/results/query_index/q01/q01.015.adm  |   1 +
 .../runtimets/results/query_index/q01/q01.016.adm  |   1 +
 .../runtimets/results/query_index/q01/q01.017.adm  |   4 +
 .../runtimets/results/query_index/q01/q01.018.adm  |   4 +
 .../runtimets/results/query_index/q01/q01.019.adm  |   4 +
 .../runtimets/results/query_index/q01/q01.020.adm  |   4 +
 .../runtimets/results/query_index/q01/q01.021.adm  |  12 ++
 .../runtimets/results/query_index/q01/q01.022.adm  |  12 ++
 .../runtimets/results/query_index/q01/q01.023.adm  |  12 ++
 .../runtimets/results/query_index/q01/q01.024.plan |  20 ++
 .../runtimets/results/query_index/q01/q01.025.plan |  20 ++
 .../runtimets/results/query_index/q01/q01.026.plan |  38 ++++
 .../runtimets/results/query_index/q01/q01.027.plan |  22 +++
 .../runtimets/results/query_index/q01/q01.028.plan |  38 ++++
 .../runtimets/results/query_index/q01/q01.029.plan |  38 ++++
 .../runtimets/results/query_index/q01/q01.030.plan |  38 ++++
 .../runtimets/results/query_index/q01/q01.031.plan |  38 ++++
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  16 ++
 .../metadata/declared/FunctionDataSource.java      |  13 +-
 .../asterix/metadata/utils/KeyFieldTypeUtil.java   |   8 +-
 .../core/algebra/metadata/IDataSource.java         |   6 +
 76 files changed, 1695 insertions(+), 48 deletions(-)

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 ecf035227c..25d42dfbc4 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.Iterator;
 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 @@ public class AccessMethodUtils {
                     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 dfe29c2b48..0b2975f921 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 @@ public abstract class FunctionRewriter implements IFunctionToDataSourceRewriter
             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 @@ public abstract class FunctionRewriter implements IFunctionToDataSourceRewriter
         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-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
similarity index 50%
copy from asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
index f5fd7dd84f..f43588e1de 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -16,32 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.metadata.declared;
+package org.apache.asterix.app.function;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 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.asterix.om.utils.RecordUtil;
 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.functions.FunctionIdentifier;
 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;
@@ -49,31 +51,46 @@ import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
-public abstract class FunctionDataSource extends DataSource {
+public class QueryIndexDatasource extends FunctionDataSource {
 
-    protected final FunctionIdentifier functionId;
+    private final Dataset ds;
+    private final String indexName;
+    private final AlgebricksAbsolutePartitionConstraint storageLocations;
 
-    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;
-        schemaTypes = new IAType[] { itemType };
+    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;
     }
 
-    public FunctionIdentifier getFunctionId() {
-        return functionId;
+    @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() {
-        return true;
+        // 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
-    public IDataSourcePropertiesProvider getPropertiesProvider() {
-        // Unordered Random partitioning on all nodes
-        return scanVariables -> new StructuralPropertiesVector(new RandomPartitioningProperty(domain),
-                Collections.emptyList());
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        throw new UnsupportedOperationException("query-index() does not use record reader adapter");
     }
 
     @Override
@@ -84,26 +101,26 @@ public abstract class FunctionDataSource extends DataSource {
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
             IProjectionInfo<?> projectionInfo) throws AlgebricksException {
-        GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
-        adapterFactory.setOutputType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-        IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
-        FunctionDataSourceFactory factory =
-                new FunctionDataSourceFactory(createFunction(metadataProvider, getLocations(csm)));
-        adapterFactory.configure(factory);
-        return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
-                tupleFilterFactory, outputLimit);
+        return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, ds, indexName,
+                null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false);
     }
 
-    protected abstract IDatasourceFunction createFunction(MetadataProvider metadataProvider,
-            AlgebricksAbsolutePartitionConstraint locations);
-
-    protected AlgebricksAbsolutePartitionConstraint getLocations(IClusterStateManager csm) {
-        String[] allPartitions = csm.getClusterLocations().getLocations();
-        Set<String> ncs = new HashSet<>(Arrays.asList(allPartitions));
-        return new AlgebricksAbsolutePartitionConstraint(ncs.toArray(new String[ncs.size()]));
+    @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);
+        };
     }
 
-    protected static DataSourceId createDataSourceId(FunctionIdentifier fid, String... parameters) {
-        return new DataSourceId(FunctionSignature.getDataverseName(fid), fid.getName(), parameters);
+    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 0000000000..1906fcf9b3
--- /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 d6dc67ca72..5a2ef3c768 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.DumpIndexRewriter;
 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 @@ public class MetadataBuiltinFunctions {
                 (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 0000000000..31934aef80
--- /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 0000000000..796f598b12
--- /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 0000000000..f5f5774455
--- /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 0000000000..9bea68e5ca
--- /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 0000000000..cf3a0e362e
--- /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 0000000000..36b2bab543
--- /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 0000000000..270a1d47d1
--- /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 0000000000..04fd66f6ef
--- /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 0000000000..1910ffc6f2
--- /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 0000000000..f801201e38
--- /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 0000000000..729a4f5144
--- /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 0000000000..2914aabbc1
--- /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 0000000000..2ed609d104
--- /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 0000000000..504cae6168
--- /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 0000000000..2272bbe9ed
--- /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 0000000000..9ddc1716d2
--- /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 0000000000..41f3505d95
--- /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 0000000000..bce624ff32
--- /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 0000000000..62c562292a
--- /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 0000000000..f4f0560dcb
--- /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 0000000000..31d5fbca5d
--- /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 0000000000..5e7caad751
--- /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 0000000000..12e1311c6f
--- /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 0000000000..7015ff7fa6
--- /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 0000000000..dc94475ec4
--- /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 0000000000..4b2379d1cf
--- /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 0000000000..ebb2269dfa
--- /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 0000000000..b535ee816e
--- /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 0000000000..87d3aadde6
--- /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 0000000000..8f6255f5f1
--- /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 0000000000..55f2b7092e
--- /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 0000000000..94b5f60aff
--- /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 0000000000..7cc70b5026
--- /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 0000000000..e6008a6859
--- /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 0000000000..361c57fd05
--- /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 0000000000..bcae31e6c1
--- /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 0000000000..4b2dc991ed
--- /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 0000000000..36b2bab543
--- /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 0000000000..1b32b5f034
--- /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 0000000000..7d4626e4c0
--- /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 0000000000..e5d1135722
--- /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 0000000000..d0d8cc8434
--- /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 0000000000..1b32b5f034
--- /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 0000000000..7d4626e4c0
--- /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 0000000000..e5d1135722
--- /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 0000000000..d0d8cc8434
--- /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 0000000000..2a57ebd387
--- /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 0000000000..0021c09d8f
--- /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 0000000000..1858896f51
--- /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 0000000000..3ce6c0902c
--- /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 0000000000..dfec480612
--- /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 0000000000..dfec480612
--- /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 0000000000..831b0cb1e0
--- /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 0000000000..6a94e7b4ff
--- /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 0000000000..6a94e7b4ff
--- /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 0000000000..6a94e7b4ff
--- /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 0000000000..3c82bb9f2d
--- /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 0000000000..8dd327fb3a
--- /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 0000000000..7cf5585efe
--- /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 0000000000..45c702870c
--- /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 0000000000..0308f76c63
--- /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 0000000000..2648cf343b
--- /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 0000000000..58e4a62ac7
--- /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 0000000000..c86f9fb9ff
--- /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 0000000000..e537bf3ab7
--- /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 0000000000..24aa7581b9
--- /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 0000000000..f2046b8024
--- /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 304520b3d7..17df68908a 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 f5fd7dd84f..a7d560c70c 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 @@ public abstract class FunctionDataSource extends DataSource {
 
     protected final FunctionIdentifier functionId;
 
-    public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain)
+    public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain, IAType itemType)
             throws AlgebricksException {
-        super(id, RecordUtil.FULLY_OPEN_RECORD_TYPE, null, DataSource.Type.FUNCTION, domain);
+        super(id, itemType, null, DataSource.Type.FUNCTION, domain);
         this.functionId = functionId;
+        initSchemaType(itemType);
+    }
+
+    public FunctionDataSource(DataSourceId id, FunctionIdentifier functionId, INodeDomain domain)
+            throws AlgebricksException {
+        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 c4cc1de05c..fd20b3c337 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 @@ public class KeyFieldTypeUtil {
      * @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 8b4e56d3fc..5b75cd9c22 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 interface IDataSource<T> {
     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();