You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2019/09/12 14:25:03 UTC

[asterixdb] 04/06: [NO ISSUE][COMP] Fix index selection for datasets with meta

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

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

commit d292d99bc652e81ae1e7877e88529ce2b29cf6d0
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Fri Sep 6 00:21:36 2019 -0700

    [NO ISSUE][COMP] Fix index selection for datasets with meta
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    Fix index selection for datasets with meta. Access method rule should
    check where the field is coming from (dataset record or meta record)
    and then determine if the field matches the keys in the index based
    on their names and sources.
    
    This patch also fixes resolving PK field accesses to the primary key
    variable (e.g. $ds.getField("id") is turned into $13 where id is a PK).
    The fix considers whether the PK is coming from the data record or the
    meta record.
    The patch also includes fixing rewriting of meta() references and
    replacing them with their corresponding meta variables. Now nested plans
    are visited also when looking for meta() references to take care of cases
    where the data scan producing the meta variable and the meta() references
    happen to be inside the nested plans.
    MetaFunctionToMetaVariable() is fired also after the rules which eliminate
    subplans to allow for rewriting of the meta() if it couldn't be replaced
    when the meta() reference was in the subplan but referring to meta variable
    outside the nested plan.
    ReinferAllTypesRule() is now fired before ByNameToByIndexFieldAccessRule()
    to allow the latter rule to get the up-to-date types in the whole plan.
    
    Change-Id: I0503f64cd51153896e2d7d7abc465c679f82e2fd
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3545
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
    Contrib: Till Westmann <ti...@apache.org>
---
 .../asterix/optimizer/base/RuleCollections.java    |  3 +-
 .../rules/MetaFunctionToMetaVariableRule.java      | 21 ++++--
 .../optimizer/rules/PushFieldAccessRule.java       | 58 ++++++++++-----
 .../am/AbstractIntroduceAccessMethodRule.java      | 82 ++++++++++++++++------
 .../optimizer/rules/am/BTreeAccessMethod.java      | 17 +++--
 .../optimizer/rules/am/IOptimizableFuncExpr.java   |  4 +-
 .../rules/am/IntroduceLSMComponentFilterRule.java  | 23 ++++--
 .../optimizer/rules/am/OptimizableFuncExpr.java    | 11 ++-
 .../rules/am/OptimizableOperatorSubTree.java       |  1 +
 asterixdb/asterix-app/data/csv/people.csv          |  9 +++
 asterixdb/asterix-app/data/csv/people3.csv         |  9 +++
 .../meta/field_access_with_data_and_meta_1.sqlpp   | 41 +++++++++++
 .../meta/field_access_with_data_and_meta_2.sqlpp   | 41 +++++++++++
 .../meta/field_access_with_data_and_meta_3.sqlpp   | 41 +++++++++++
 .../meta/field_access_with_data_and_meta_4.sqlpp   | 41 +++++++++++
 .../meta/indexes_on_dataset_with_meta_01.sqlpp     | 44 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_02.sqlpp     | 44 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_03.sqlpp     | 44 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_04.sqlpp     | 44 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_05.sqlpp     | 48 +++++++++++++
 .../meta/indexes_on_dataset_with_meta_06.sqlpp     | 48 +++++++++++++
 .../meta/indexes_on_dataset_with_meta_07.sqlpp     | 48 +++++++++++++
 .../meta/indexes_on_dataset_with_meta_08.sqlpp     | 46 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_09.sqlpp     | 45 ++++++++++++
 .../meta/indexes_on_dataset_with_meta_10.sqlpp     | 47 +++++++++++++
 .../meta/indexes_on_dataset_with_meta_11.sqlpp     | 47 +++++++++++++
 .../queries/meta/with_clause_meta.sqlpp            | 47 +++++++++++++
 .../meta/field_access_with_data_and_meta_1.plan    | 14 ++++
 .../meta/field_access_with_data_and_meta_2.plan    | 11 +++
 .../meta/field_access_with_data_and_meta_3.plan    | 11 +++
 .../meta/field_access_with_data_and_meta_4.plan    | 14 ++++
 .../meta/indexes_on_dataset_with_meta_01.plan      | 10 +++
 .../meta/indexes_on_dataset_with_meta_02.plan      | 10 +++
 .../meta/indexes_on_dataset_with_meta_03.plan      | 10 +++
 .../meta/indexes_on_dataset_with_meta_04.plan      | 11 +++
 .../meta/indexes_on_dataset_with_meta_05.plan      | 24 +++++++
 .../meta/indexes_on_dataset_with_meta_06.plan      | 17 +++++
 .../meta/indexes_on_dataset_with_meta_07.plan      | 18 +++++
 .../meta/indexes_on_dataset_with_meta_08.plan      | 17 +++++
 .../meta/indexes_on_dataset_with_meta_09.plan      | 10 +++
 .../meta/indexes_on_dataset_with_meta_10.plan      | 24 +++++++
 .../meta/indexes_on_dataset_with_meta_11.plan      | 17 +++++
 .../optimizerts/results/meta/with_clause_meta.plan | 44 ++++++++++++
 .../dataset-with-meta.01.ddl.sqlpp                 | 68 ++++++++++++++++++
 .../dataset-with-meta.02.update.sqlpp              | 27 +++++++
 .../dataset-with-meta.03.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.04.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.05.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.06.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.07.query.sqlpp               | 30 ++++++++
 .../dataset-with-meta.08.query.sqlpp               | 30 ++++++++
 .../dataset-with-meta.09.query.sqlpp               | 30 ++++++++
 .../dataset-with-meta.10.ddl.sqlpp                 | 22 ++++++
 .../dataset-with-meta.11.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.12.query.sqlpp               | 29 ++++++++
 .../dataset-with-meta.13.query.sqlpp               | 30 ++++++++
 .../dataset-with-meta.14.query.sqlpp               | 30 ++++++++
 .../dataset-with-meta.15.ddl.sqlpp                 | 20 ++++++
 .../meta_in_with_clause.1.ddl.sqlpp                | 71 +++++++++++++++++++
 .../meta_in_with_clause.2.update.sqlpp             | 27 +++++++
 .../meta_in_with_clause.3.query.sqlpp              | 29 ++++++++
 .../meta_in_with_clause.4.query.sqlpp              | 29 ++++++++
 .../meta_in_with_clause.5.query.sqlpp              | 29 ++++++++
 .../meta_in_with_clause.6.query.sqlpp              | 29 ++++++++
 .../meta_in_with_clause.7.ddl.sqlpp                | 20 ++++++
 .../resolving_pk_with_meta.1.ddl.sqlpp             | 68 ++++++++++++++++++
 .../resolving_pk_with_meta.2.update.sqlpp          | 27 +++++++
 .../resolving_pk_with_meta.3.query.sqlpp           | 27 +++++++
 .../resolving_pk_with_meta.4.query.sqlpp           | 27 +++++++
 .../resolving_pk_with_meta.5.query.sqlpp           | 27 +++++++
 .../resolving_pk_with_meta.6.query.sqlpp           | 27 +++++++
 .../resolving_pk_with_meta.7.ddl.sqlpp             | 27 +++++++
 .../dataset-with-meta/dataset-with-meta.03.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.04.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.05.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.06.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.07.adm     |  9 +++
 .../dataset-with-meta/dataset-with-meta.08.adm     |  2 +
 .../dataset-with-meta/dataset-with-meta.09.adm     |  9 +++
 .../dataset-with-meta/dataset-with-meta.11.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.12.adm     |  1 +
 .../dataset-with-meta/dataset-with-meta.13.adm     |  9 +++
 .../dataset-with-meta/dataset-with-meta.14.adm     |  2 +
 .../meta_in_with_clause/meta_in_with_clause.3.adm  |  8 +++
 .../meta_in_with_clause/meta_in_with_clause.4.adm  |  8 +++
 .../meta_in_with_clause/meta_in_with_clause.5.adm  |  8 +++
 .../meta_in_with_clause/meta_in_with_clause.6.adm  |  8 +++
 .../resolving_pk_with_meta.3.adm                   |  9 +++
 .../resolving_pk_with_meta.4.adm                   |  9 +++
 .../resolving_pk_with_meta.5.adm                   |  9 +++
 .../resolving_pk_with_meta.6.adm                   |  9 +++
 .../test/resources/runtimets/testsuite_sqlpp.xml   | 17 +++++
 .../apache/asterix/metadata/utils/DatasetUtil.java | 41 ++++++++++-
 93 files changed, 2247 insertions(+), 57 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index f9b5c38..841043d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -235,7 +235,7 @@ public final class RuleCollections {
         // The following rule should be fired after PushAggregateIntoNestedSubplanRule because
         // pulling invariants out of a subplan will make PushAggregateIntoGroupby harder.
         condPushDownAndJoinInference.add(new AsterixMoveFreeVariableOperatorOutOfSubplanRule());
-
+        condPushDownAndJoinInference.add(new MetaFunctionToMetaVariableRule());
         return condPushDownAndJoinInference;
     }
 
@@ -244,6 +244,7 @@ public final class RuleCollections {
         fieldLoads.add(new LoadRecordFieldsRule());
         fieldLoads.add(new PushFieldAccessRule());
         // fieldLoads.add(new ByNameToByHandleFieldAccessRule()); -- disabled
+        fieldLoads.add(new ReinferAllTypesRule());
         fieldLoads.add(new ByNameToByIndexFieldAccessRule());
         fieldLoads.add(new RemoveRedundantVariablesRule());
         fieldLoads.add(new AsterixInlineVariablesRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index cd99c28..c47acd1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -34,6 +34,7 @@ import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 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.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
@@ -44,6 +45,8 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCall
 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.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -148,6 +151,15 @@ public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
             }
             currentTransformer = new CompositeExpressionReferenceTransform(transformers);
         }
+
+        if (((AbstractLogicalOperator) op).hasNestedPlans()) {
+            AbstractOperatorWithNestedPlans opWithNestedPlans = (AbstractOperatorWithNestedPlans) op;
+            for (ILogicalPlan nestedPlan : opWithNestedPlans.getNestedPlans()) {
+                for (Mutable<ILogicalOperator> root : nestedPlan.getRoots()) {
+                    visit(root);
+                }
+            }
+        }
         rewritten |= op.acceptExpressionTransform(currentTransformer);
         return currentTransformer;
     }
@@ -197,13 +209,14 @@ class LogicalExpressionReferenceTransform implements ILogicalExpressionReference
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         List<Mutable<ILogicalExpression>> argRefs = funcExpr.getArguments();
 
+        boolean changed = false;
         // Recursively transform argument expressions.
         for (Mutable<ILogicalExpression> argRef : argRefs) {
-            transform(argRef);
+            changed |= transform(argRef);
         }
 
         if (!funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.META)) {
-            return false;
+            return changed;
         }
         // The user query provides more than one parameter for the meta function.
         if (argRefs.size() > 1) {
@@ -215,12 +228,12 @@ class LogicalExpressionReferenceTransform implements ILogicalExpressionReference
         if (argRefs.size() == 1) {
             ILogicalExpression argExpr = argRefs.get(0).getValue();
             if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                return false;
+                return changed;
             }
             VariableReferenceExpression argVarExpr = (VariableReferenceExpression) argExpr;
             LogicalVariable argVar = argVarExpr.getVariableReference();
             if (!dataVar.equals(argVar)) {
-                return false;
+                return changed;
             }
             VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
             metaVarRef.setSourceLocation(expr.getSourceLocation());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index b779c83..758b041 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -288,8 +288,6 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
             // fields. If yes, we can equate the two variables.
             if (op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
                 DataSourceScanOperator scan = (DataSourceScanOperator) op2;
-                int n = scan.getVariables().size();
-                LogicalVariable scanRecordVar = scan.getVariables().get(n - 1);
 
                 IDataSource<DataSourceId> dataSource = (IDataSource<DataSourceId>) scan.getDataSource();
                 byte dsType = ((DataSource) dataSource).getDatasourceType();
@@ -308,6 +306,10 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
                     return false;
                 }
 
+                List<LogicalVariable> allVars = scan.getVariables();
+                LogicalVariable dataRecVarInScan = ((DataSource) dataSource).getDataRecordVariable(allVars);
+                LogicalVariable metaRecVarInScan = ((DataSource) dataSource).getMetaVariable(allVars);
+
                 String tName = dataset.getItemTypeName();
                 IAType t = mp.findType(dataset.getItemTypeDataverseName(), tName);
                 if (t.getTypeTag() != ATypeTag.OBJECT) {
@@ -315,23 +317,37 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
                 }
                 ARecordType rt = (ARecordType) t;
                 Pair<ILogicalExpression, List<String>> fieldPathAndVar = getFieldExpression(access, rt);
-                ILogicalExpression e0 = fieldPathAndVar.first;
-                LogicalExpressionTag tag = e0.getExpressionTag();
-                if (tag == LogicalExpressionTag.VARIABLE) {
-                    VariableReferenceExpression varRef = (VariableReferenceExpression) e0;
-                    if (varRef.getVariableReference() == scanRecordVar) {
-                        int p = DatasetUtil.getPositionOfPartitioningKeyField(dataset, fieldPathAndVar.second);
-                        if (p < 0) { // not one of the partitioning fields
-                            setAsFinal(access, context, finalAnnot);
-                            return false;
+                ILogicalExpression targetRecVar = fieldPathAndVar.first;
+                List<String> targetFieldPath = fieldPathAndVar.second;
+                boolean rewrite = false;
+                boolean fieldFromMeta = false;
+                if (sameRecords(targetRecVar, dataRecVarInScan)) {
+                    rewrite = true;
+                } else {
+                    // check meta part
+                    IAType metaType = mp.findMetaType(dataset); // could be null
+                    if (metaType != null && metaType.getTypeTag() == ATypeTag.OBJECT) {
+                        fieldPathAndVar = getFieldExpression(access, (ARecordType) metaType);
+                        targetRecVar = fieldPathAndVar.first;
+                        targetFieldPath = fieldPathAndVar.second;
+                        if (sameRecords(targetRecVar, metaRecVarInScan)) {
+                            rewrite = true;
+                            fieldFromMeta = true;
                         }
-                        LogicalVariable keyVar = scan.getVariables().get(p);
-                        VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
-                        keyVarRef.setSourceLocation(varRef.getSourceLocation());
-                        access.getExpressions().get(0).setValue(keyVarRef);
-                        return true;
+                    }
+                }
 
+                if (rewrite) {
+                    int p = DatasetUtil.getPositionOfPartitioningKeyField(dataset, targetFieldPath, fieldFromMeta);
+                    if (p < 0) { // not one of the partitioning fields
+                        setAsFinal(access, context, finalAnnot);
+                        return false;
                     }
+                    LogicalVariable keyVar = scan.getVariables().get(p);
+                    VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                    keyVarRef.setSourceLocation(targetRecVar.getSourceLocation());
+                    access.getExpressions().get(0).setValue(keyVarRef);
+                    return true;
                 }
             }
             setAsFinal(access, context, finalAnnot);
@@ -339,6 +355,16 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         }
     }
 
+    /**
+     * @param recordInAssign the variable reference expression in assign op
+     * @param recordInScan the record (payload) variable in scan op
+     * @return true if the expression in the assign op is a variable and that variable = record variable in scan op
+     */
+    private boolean sameRecords(ILogicalExpression recordInAssign, LogicalVariable recordInScan) {
+        return recordInAssign != null && recordInAssign.getExpressionTag() == LogicalExpressionTag.VARIABLE
+                && ((VariableReferenceExpression) recordInAssign).getVariableReference().equals(recordInScan);
+    }
+
     private Pair<ILogicalExpression, List<String>> getFieldExpression(AssignOperator access, ARecordType rt)
             throws AlgebricksException {
         LinkedList<String> fieldPath = new LinkedList<>();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index c8bec85..d232ce0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -34,7 +34,9 @@ import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 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.MetadataUtil;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
@@ -51,6 +53,7 @@ import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
 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;
@@ -533,17 +536,20 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      */
     protected boolean fillIndexExprs(List<Index> datasetIndexes, List<String> fieldName, IAType fieldType,
             IOptimizableFuncExpr optFuncExpr, int matchedFuncExprIndex, int varIdx,
-            OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx)
+            OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx, int fieldSource)
             throws AlgebricksException {
         List<Index> indexCandidates = new ArrayList<>();
         // Add an index to the candidates if one of the indexed fields is fieldName
         for (Index index : datasetIndexes) {
             // Need to also verify the index is pending no op
-            if (index.getKeyFieldNames().contains(fieldName) && index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
+            int keyIdx = index.getKeyFieldNames().indexOf(fieldName);
+            List<Integer> keySources = index.getKeyFieldSourceIndicators();
+            if (keyIdx >= 0 && keySourceMatches(keySources, keyIdx, fieldSource)
+                    && index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
                 indexCandidates.add(index);
                 boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
                 if (isFieldTypeUnknown && (!index.isOverridingKeyFieldTypes() || index.isEnforced())) {
-                    IAType indexedType = index.getKeyFieldTypes().get(index.getKeyFieldNames().indexOf(fieldName));
+                    IAType indexedType = index.getKeyFieldTypes().get(keyIdx);
                     optFuncExpr.setFieldType(varIdx, indexedType);
                 }
                 analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
@@ -556,6 +562,11 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         return true;
     }
 
+    private static boolean keySourceMatches(List<Integer> keySources, int keyIdx, int fieldSource) {
+        // TODO(ali): keySources from Index should not be null. should investigate if it can happen (ie on external ds)
+        return keySources == null ? fieldSource == 0 : keySources.get(keyIdx) == fieldSource;
+    }
+
     protected void fillAllIndexExprs(OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context) throws AlgebricksException {
         int optFuncExprIndex = 0;
@@ -624,6 +635,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         // Remember matching subtree.
         optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
         List<String> fieldName = null;
+        MutableInt fieldSource = new MutableInt(0);
         if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
             VariableReferenceExpression varRef = new VariableReferenceExpression(var);
             varRef.setSourceLocation(unnestOp.getSourceLocation());
@@ -631,7 +643,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         } else {
             fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
                     funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
-                    subTree.getMetaRecordType(), datasetMetaVar);
+                    subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
             if (fieldName.isEmpty()) {
                 return;
             }
@@ -640,13 +652,13 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 (IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
         // Set the fieldName in the corresponding matched function
         // expression.
-        optFuncExpr.setFieldName(funcVarIndex, fieldName);
+        optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource.intValue());
         optFuncExpr.setFieldType(funcVarIndex, fieldType);
 
         setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
         if (subTree.hasDataSource()) {
             fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex, subTree,
-                    analysisCtx);
+                    analysisCtx, fieldSource.intValue());
         }
     }
 
@@ -655,6 +667,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
             AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         List<LogicalVariable> varList = assignOp.getVariables();
+        MutableInt fieldSource = new MutableInt(0);
         for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
             LogicalVariable var = varList.get(varIndex);
             int optVarIndex = optFuncExpr.findLogicalVar(var);
@@ -667,21 +680,22 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             // Remember matching subtree.
             optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
 
+            fieldSource.setValue(0);
             List<String> fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
                     subTree.getRecordType(), optVarIndex,
                     optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), subTree.getMetaRecordType(),
-                    datasetMetaVar);
+                    datasetMetaVar, fieldSource);
 
             IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
             // Set the fieldName in the corresponding matched
             // function expression.
-            optFuncExpr.setFieldName(optVarIndex, fieldName);
+            optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
             optFuncExpr.setFieldType(optVarIndex, fieldType);
 
             setTypeTag(context, subTree, optFuncExpr, optVarIndex);
             if (subTree.hasDataSource()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, optVarIndex,
-                        subTree, analysisCtx);
+                        subTree, analysisCtx, fieldSource.intValue());
             }
         }
     }
@@ -690,6 +704,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             List<Index> datasetIndexes, List<LogicalVariable> dsVarList, OptimizableOperatorSubTree subTree,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean fromAdditionalDataSource)
             throws AlgebricksException {
+        MutableInt mutableFieldSource = new MutableInt(0);
         for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
             LogicalVariable var = dsVarList.get(varIndex);
             int funcVarIndex = optFuncExpr.findLogicalVar(var);
@@ -701,28 +716,39 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             List<String> fieldName = null;
             IAType fieldType = null;
             List<List<String>> subTreePKs = null;
+            mutableFieldSource.setValue(0);
 
             if (!fromAdditionalDataSource) {
-                subTreePKs = subTree.getDataset().getPrimaryKeys();
+                Dataset dataset = subTree.getDataset();
+                subTreePKs = dataset.getPrimaryKeys();
                 // Check whether this variable is PK, not a record variable.
                 if (varIndex <= subTreePKs.size() - 1) {
                     fieldName = subTreePKs.get(varIndex);
                     fieldType = (IAType) context.getOutputTypeEnvironment(subTree.getDataSourceRef().getValue())
                             .getVarType(var);
+                    List<Integer> keySourceIndicators = DatasetUtil.getKeySourceIndicators(dataset);
+                    if (keySourceIndicators != null) {
+                        mutableFieldSource.setValue(keySourceIndicators.get(varIndex));
+                    }
                 }
             } else {
                 // Need to check additional dataset one by one
                 for (int i = 0; i < subTree.getIxJoinOuterAdditionalDatasets().size(); i++) {
-                    if (subTree.getIxJoinOuterAdditionalDatasets().get(i) != null) {
-                        subTreePKs = subTree.getIxJoinOuterAdditionalDatasets().get(i).getPrimaryKeys();
-
+                    Dataset dataset = subTree.getIxJoinOuterAdditionalDatasets().get(i);
+                    if (dataset != null) {
+                        subTreePKs = dataset.getPrimaryKeys();
                         // Check whether this variable is PK, not a record variable.
+                        // TODO(ali): investigate why var (LogicalVariable) is looked up in subTreePKs (List<List<str>>)
                         if (subTreePKs.contains(var) && varIndex <= subTreePKs.size() - 1) {
                             fieldName = subTreePKs.get(varIndex);
                             fieldType = (IAType) context
                                     .getOutputTypeEnvironment(
                                             subTree.getIxJoinOuterAdditionalDataSourceRefs().get(i).getValue())
                                     .getVarType(var);
+                            List<Integer> keySourceIndicators = DatasetUtil.getKeySourceIndicators(dataset);
+                            if (keySourceIndicators != null) {
+                                mutableFieldSource.setValue(keySourceIndicators.get(varIndex));
+                            }
                             break;
                         }
                     }
@@ -730,7 +756,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             }
             // Set the fieldName in the corresponding matched function
             // expression, and remember matching subtree.
-            optFuncExpr.setFieldName(funcVarIndex, fieldName);
+            int fieldSource = mutableFieldSource.intValue();
+            optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource);
             optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
             optFuncExpr.setSourceVar(funcVarIndex, var);
             VariableReferenceExpression varRef = new VariableReferenceExpression(var);
@@ -739,7 +766,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
             if (subTree.hasDataSourceScan()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
-                        subTree, analysisCtx);
+                        subTree, analysisCtx, fieldSource);
             }
         }
     }
@@ -761,7 +788,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
      */
     protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
             int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
-            ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar)
+            ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar, MutableInt fieldSource)
             throws AlgebricksException {
         // Get expression corresponding to opVar at varIndex.
         AbstractLogicalExpression expr = null;
@@ -830,6 +857,11 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         if (isFieldAccess) {
             LogicalVariable sourceVar =
                     ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
+            if (sourceVar.equals(metaVar)) {
+                fieldSource.setValue(1);
+            } else {
+                fieldSource.setValue(0);
+            }
             if (optFuncExpr != null) {
                 optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
             }
@@ -870,7 +902,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 //Recursive call on nested assign
                 List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
                         assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
-                        parentFuncExpr, metaType, metaVar);
+                        parentFuncExpr, metaType, metaVar, fieldSource);
 
                 if (parentFieldNames.isEmpty()) {
                     //Nested assign was not a field access.
@@ -946,7 +978,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     if (var.equals(curVar) && optFuncExpr != null) {
                         optFuncExpr.setSourceVar(funcVarIndex, var);
                         return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
-                                funcVarIndex, childFuncExpr, metaType, metaVar);
+                                funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource);
                     }
                 }
             } else {
@@ -954,7 +986,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 LogicalVariable var = unnestOp.getVariable();
                 if (var.equals(curVar)) {
                     getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
-                            childFuncExpr, metaType, metaVar);
+                            childFuncExpr, metaType, metaVar, fieldSource);
                 }
             }
         }
@@ -973,6 +1005,7 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
             }
         }
+        MutableInt fieldSource = new MutableInt(0);
         for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
                 .size(); assignOrUnnestIndex++) {
             AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
@@ -983,8 +1016,10 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    fieldSource.setValue(0);
                     List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
+                            fieldSource);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -996,8 +1031,9 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                 if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    fieldSource.setValue(0);
                     fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
-                            -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                            -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1022,8 +1058,10 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
+                    fieldSource.setValue(0);
                     List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar);
+                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
+                            fieldSource);
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index b45e1c6..8845722 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -350,10 +350,12 @@ public class BTreeAccessMethod implements IAccessMethod {
         for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
             // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
             IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprIndex.first);
-            int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
+            int keyPos = indexOf(optFuncExpr.getFieldName(0), optFuncExpr.getFieldSource(0),
+                    chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
             if (keyPos < 0 && optFuncExpr.getNumLogicalVars() > 1) {
                 // If we are optimizing a join, the matching field may be the second field name.
-                keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
+                keyPos = indexOf(optFuncExpr.getFieldName(1), optFuncExpr.getFieldSource(1),
+                        chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
             }
             if (keyPos < 0) {
                 throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR,
@@ -817,10 +819,11 @@ public class BTreeAccessMethod implements IAccessMethod {
         }
     }
 
-    private <T> int indexOf(T value, List<T> coll) {
+    private static int indexOf(List<String> fieldName, int fieldSource, List<List<String>> keyNames,
+            List<Integer> keySources) {
         int i = 0;
-        for (T member : coll) {
-            if (member.equals(value)) {
+        for (List<String> keyName : keyNames) {
+            if (keyName.equals(fieldName) && keyMatches(keySources, i, fieldSource)) {
                 return i;
             }
             i++;
@@ -828,6 +831,10 @@ public class BTreeAccessMethod implements IAccessMethod {
         return -1;
     }
 
+    private static boolean keyMatches(List<Integer> keySources, int keyIndex, int fieldSource) {
+        return keySources == null ? fieldSource == 0 : keySources.get(keyIndex) == fieldSource;
+    }
+
     private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree probeSubTree) {
         ComparisonKind ck =
                 AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
index 05dc4a6..6278865 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
@@ -43,10 +43,12 @@ public interface IOptimizableFuncExpr {
 
     public ILogicalExpression getLogicalExpr(int index);
 
-    public void setFieldName(int index, List<String> fieldName);
+    public void setFieldName(int index, List<String> fieldName, int fieldSource);
 
     public List<String> getFieldName(int index);
 
+    public int getFieldSource(int index);
+
     public void setFieldType(int index, IAType fieldName);
 
     public IAType getFieldType(int index);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 7739833..bfae365 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -117,7 +117,9 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
             for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
                 IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
                 boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, recType, datasetIndexes, context);
-                if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)) {
+                // the field name source should be from the dataset record, i.e. source should be == 0
+                if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)
+                        && optFuncExpr.getFieldSource(0) == 0) {
                     optFuncExprs.add(optFuncExpr);
                 }
             }
@@ -500,12 +502,13 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                     if (funcVarIndex == -1) {
                         continue;
                     }
+                    // TODO(ali): this SQ NPE should be investigated
                     List<String> fieldName =
                             getFieldNameFromSubAssignTree(optFuncExpr, descendantOp, varIndex, recType).second;
                     if (fieldName == null) {
                         return false;
                     }
-                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName, 0);
                     return true;
                 }
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
@@ -522,7 +525,9 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                     if (fieldName == null) {
                         return false;
                     }
-                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    List<Integer> keySourceIndicators = DatasetUtil.getKeySourceIndicators(dataset);
+                    int keySource = getKeySource(keySourceIndicators, varIndex);
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName, keySource);
                     return true;
                 }
             } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
@@ -560,15 +565,19 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                     ARecordType metaRecType = (ARecordType) metaItemType;
                     int numSecondaryKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, recType, metaRecType);
                     List<String> fieldName;
+                    int keySource;
                     if (varIndex >= numSecondaryKeys) {
-                        fieldName = dataset.getPrimaryKeys().get(varIndex - numSecondaryKeys);
+                        int idx = varIndex - numSecondaryKeys;
+                        fieldName = dataset.getPrimaryKeys().get(idx);
+                        keySource = getKeySource(DatasetUtil.getKeySourceIndicators(dataset), idx);
                     } else {
                         fieldName = index.getKeyFieldNames().get(varIndex);
+                        keySource = getKeySource(index.getKeyFieldSourceIndicators(), varIndex);
                     }
                     if (fieldName == null) {
                         return false;
                     }
-                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName, keySource);
                     return true;
                 }
             }
@@ -581,6 +590,10 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
         return false;
     }
 
+    private static int getKeySource(List<Integer> keySourceIndicators, int keyIdx) {
+        return keySourceIndicators == null ? 0 : keySourceIndicators.get(keyIdx);
+    }
+
     private Pair<ARecordType, List<String>> getFieldNameFromSubAssignTree(IOptimizableFuncExpr optFuncExpr,
             AbstractLogicalOperator op, int varIndex, ARecordType recType) {
         AbstractLogicalExpression expr = null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
index 1389f5e..7b04340 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
@@ -37,6 +37,7 @@ public class OptimizableFuncExpr implements IOptimizableFuncExpr {
     protected final LogicalVariable[] sourceVars;
     protected final ILogicalExpression[] logicalExprs;
     protected final List<List<String>> fieldNames;
+    protected final int[] fieldSources;
     protected final IAType[] fieldTypes;
     protected final OptimizableOperatorSubTree[] subTrees;
     protected final ILogicalExpression[] constantExpressions;
@@ -51,6 +52,7 @@ public class OptimizableFuncExpr implements IOptimizableFuncExpr {
         this.logicalExprs = new ILogicalExpression[logicalVars.length];
         this.constantExpressionTypes = constantExpressionTypes;
         this.constantExpressions = constantExpressions;
+        this.fieldSources = new int[logicalVars.length];
         this.fieldNames = new ArrayList<List<String>>();
         for (int i = 0; i < logicalVars.length; i++) {
             fieldNames.add(new ArrayList<String>());
@@ -74,6 +76,7 @@ public class OptimizableFuncExpr implements IOptimizableFuncExpr {
         this.logicalExprs = new ILogicalExpression[1];
         this.constantExpressions = new ILogicalExpression[] { constantExpression };
         this.constantExpressionTypes = new IAType[] { constantExpressionType };
+        this.fieldSources = new int[logicalVars.length];
         this.fieldNames = new ArrayList<List<String>>();
         for (int i = 0; i < logicalVars.length; i++) {
             fieldNames.add(new ArrayList<String>());
@@ -118,8 +121,9 @@ public class OptimizableFuncExpr implements IOptimizableFuncExpr {
     }
 
     @Override
-    public void setFieldName(int index, List<String> fieldName) {
+    public void setFieldName(int index, List<String> fieldName, int fieldSource) {
         fieldNames.set(index, fieldName);
+        fieldSources[index] = fieldSource;
     }
 
     @Override
@@ -128,6 +132,11 @@ public class OptimizableFuncExpr implements IOptimizableFuncExpr {
     }
 
     @Override
+    public int getFieldSource(int index) {
+        return fieldSources[index];
+    }
+
+    @Override
     public void setFieldType(int index, IAType fieldType) {
         fieldTypes[index] = fieldType;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 9673da3..578b516 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -80,6 +80,7 @@ public class OptimizableOperatorSubTree {
     private ARecordType metaRecordType = null;
     // Contains the field names for all assign operations in this sub-tree.
     // This will be used for the index-only plan check.
+    // TODO(ali): this map should be fixed to include the source of the field (dataset record or meta record)
     private Map<LogicalVariable, List<String>> varsToFieldNameMap = new HashMap<>();
 
     // Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
diff --git a/asterixdb/asterix-app/data/csv/people.csv b/asterixdb/asterix-app/data/csv/people.csv
new file mode 100644
index 0000000..9b009ac
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/people.csv
@@ -0,0 +1,9 @@
+76041664,"{""id"":1, ""name"":""John Mad"", ""age"":29, ""hobby"":""reading""}"
+52037425,"{""id"":2, ""name"":""Scott Scott"", ""age"":30, ""hobby"":""hiking""}"
+45962603,"{""id"":3, ""name"":""Dan David"", ""age"":40, ""hobby"":""bowling""}"
+51041435,"{""id"":4, ""name"":""Robert Moore"", ""age"":32, ""hobby"":""reading""}"
+39225791,"{""id"":5, ""name"":""Sandy Donald"", ""age"":35, ""hobby"":""soccer""}"
+13071782,"{""id"":6, ""name"":""Joe Dana"", ""age"":24, ""hobby"":""tennis""}"
+26237702,"{""id"":7, ""name"":""Watson Jordon"", ""age"":28, ""hobby"":""basketball""}"
+32571888,"{""id"":8, ""name"":""Mat Steve"", ""age"":45, ""hobby"":""tennis""}"
+86897761,"{""id"":9, ""name"":""Sandra Pec"", ""age"":36, ""hobby"":""hiking""}"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/people3.csv b/asterixdb/asterix-app/data/csv/people3.csv
new file mode 100644
index 0000000..817b4a6
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/people3.csv
@@ -0,0 +1,9 @@
+76041664,"{""id"":""1"", ""name"":""John Mad"", ""age"":29, ""hobby"":""reading""}"
+52037425,"{""id"":""2"", ""name"":""Scott Scott"", ""age"":30, ""hobby"":""hiking""}"
+45962603,"{""id"":""3"", ""name"":""Dan David"", ""age"":40, ""hobby"":""bowling""}"
+51041435,"{""id"":""4"", ""name"":""Robert Moore"", ""age"":32, ""hobby"":""reading""}"
+7,"{""id"":""5"", ""name"":""Sandy Donald"", ""age"":35, ""hobby"":""soccer""}"
+13071782,"{""id"":""6"", ""name"":""Joe Dana"", ""age"":24, ""hobby"":""tennis""}"
+5,"{""id"":""7"", ""name"":""Watson Jordon"", ""age"":28, ""hobby"":""basketball""}"
+32571888,"{""id"":""8"", ""name"":""Mat Steve"", ""age"":45, ""hobby"":""tennis""}"
+86897761,"{""id"":""9"", ""name"":""Sandra Pec"", ""age"":36, ""hobby"":""hiking""}"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_1.sqlpp
new file mode 100644
index 0000000..2c7a94e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_1.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: tests that meta().id isn't resolved as PK when the primary key happens to have the same name as the meta.
+ *              Data scan happens.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MetaType as open {
+  id:int32
+};
+
+create type DataType as open {
+  id:int32,
+  text: string
+};
+
+create dataset ds(DataType) with meta(MetaType) primary key id;
+create primary index on ds;
+
+select count(meta().id) from ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_2.sqlpp
new file mode 100644
index 0000000..74d39ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_2.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: tests that primary key access is resolved correctly in the presence of meta having the same field name.
+ *              Primary index is used
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MetaType as open {
+  id:int32
+};
+
+create type DataType as open {
+  id:int32,
+  text: string
+};
+
+create dataset ds(DataType) with meta(MetaType) primary key id;
+create primary index on ds;
+
+select count(id) from ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_3.sqlpp
new file mode 100644
index 0000000..453f5b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_3.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: tests that meta().id is resolved correctly as PK in the presence of a data record field having the same name.
+ *              Primary index is used.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MetaType as open {
+  id:int32
+};
+
+create type DataType as open {
+  id:int32,
+  text: string
+};
+
+create dataset ds(DataType) with meta(MetaType) primary key meta().id;
+create primary index on ds;
+
+select count(meta().id) from ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_4.sqlpp
new file mode 100644
index 0000000..722cc4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_4.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: tests that id isn't resolved as PK when the PK is coming from the meta record having the same name.
+ *              Data scan happens.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MetaType as open {
+  id:int32
+};
+
+create type DataType as open {
+  id:int32,
+  text: string
+};
+
+create dataset ds(DataType) with meta(MetaType) primary key meta().id;
+create primary index on ds;
+
+select count(id) from ds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_01.sqlpp
new file mode 100644
index 0000000..7955726
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_01.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that data scan is used (no primary index is used)
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+
+// DS1 primary index is on meta().id
+select * from DS1
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_02.sqlpp
new file mode 100644
index 0000000..398c028
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_02.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+
+// DS1 primary index is on meta().id
+select * from DS1
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_03.sqlpp
new file mode 100644
index 0000000..a701d86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_03.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+// DS2 primary index is on id
+select * from DS2
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_04.sqlpp
new file mode 100644
index 0000000..a63697a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_04.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that data scan is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+// DS2 primary index is on id
+select * from DS2
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_05.sqlpp
new file mode 100644
index 0000000..67d016e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_05.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that data scan is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = ds1.id
+select ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_06.sqlpp
new file mode 100644
index 0000000..8a72f86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_06.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS1 is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = meta(ds1).id
+select meta(ds1).id as ds1_meta_id, ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_07.sqlpp
new file mode 100644
index 0000000..9753c9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_07.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS2 is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS1 as ds1 join DS2 as ds2 on ds1.id /*+ indexnl */ = ds2.id
+select ds1, ds2
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_08.sqlpp
new file mode 100644
index 0000000..5540b38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_08.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that secondary index is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create index id_sec_idx on DS1(id);
+
+// DS1 primary index is on meta().id & now has a secondary index on id
+
+select * from DS1
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_09.sqlpp
new file mode 100644
index 0000000..27b69db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_09.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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create index id_sec_idx on DS1(id);
+
+// DS1 primary index is on meta().id & now has a secondary index on id
+select * from DS1
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_10.sqlpp
new file mode 100644
index 0000000..68cee62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_10.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that secondary index of DS1 is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+create index id_sec_idx on DS1(id);
+
+// DS1 primary index is on meta().id & now has a secondary index on id. DS2 primary index is on id
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = ds1.id
+select ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_11.sqlpp
new file mode 100644
index 0000000..d5b922a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/indexes_on_dataset_with_meta_11.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS1 is used
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+create index id_sec_idx on DS1(id);
+
+// DS1 primary index is on meta().id & now has a secondary index on id. DS2 primary index is on id
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = meta(ds1).id
+select meta(ds1).id as ds1_meta_id, ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/with_clause_meta.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/with_clause_meta.sqlpp
new file mode 100644
index 0000000..dc94800
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/with_clause_meta.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: tests that meta().id is resolved as PK when used in WITH clause.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MetaType as open {
+  id:string
+};
+
+create type DataType as open {
+  id:int32,
+  text: string
+};
+
+create dataset ds(DataType) with meta(MetaType) primary key id;
+
+with raw_data as (
+  select meta(a).id as aid, age as age
+  from ds a
+  where hobby in ["tennis", "reading", "hiking", "soccer"] )
+
+select t2.raw_data.age, t2.raw_data.aid, array_count(t1) total
+from (select value g from raw_data group by series group as g) as t1 unnest t1 as t2
+order by aid
+limit 5;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
new file mode 100644
index 0000000..000f2f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
new file mode 100644
index 0000000..efb6c63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
new file mode 100644
index 0000000..efb6c63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
new file mode 100644
index 0000000..000f2f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
@@ -0,0 +1,14 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan
new file mode 100644
index 0000000..5df1890
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_01.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan
new file mode 100644
index 0000000..fec009a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_02.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan
new file mode 100644
index 0000000..fec009a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_03.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan
new file mode 100644
index 0000000..1a11a0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_04.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan
new file mode 100644
index 0000000..83d3362
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$25][$$30]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
new file mode 100644
index 0000000..2273bfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan
new file mode 100644
index 0000000..6e0b4cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan
new file mode 100644
index 0000000..04dcc66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_08.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan
new file mode 100644
index 0000000..fec009a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_09.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan
new file mode 100644
index 0000000..b518b83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_10.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
new file mode 100644
index 0000000..2273bfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
new file mode 100644
index 0000000..ae0fb17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 5] [$#1(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- UNNEST  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$95]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$76][$$99]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- UNNEST  |UNPARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.01.ddl.sqlpp
new file mode 100644
index 0000000..ae60888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.01.ddl.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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 DataType as open {
+id:string,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string
+};
+
+create dataset DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+create feed DsStream1 with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people3.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header" : "false"
+};
+
+create feed DsStream2 with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people3.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "0",
+ "header" : "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.02.update.sqlpp
new file mode 100644
index 0000000..3174cc5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.02.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 `wait-for-completion-feed` "true";
+connect feed DsStream1 to dataset DS1;
+start feed DsStream1;
+
+connect feed DsStream2 to dataset DS2;
+start feed DsStream2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.03.query.sqlpp
new file mode 100644
index 0000000..39e3707
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.03.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that data scan is used (no primary index is used)
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id
+use test;
+
+select * from DS1
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.04.query.sqlpp
new file mode 100644
index 0000000..cb87da7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.04.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id
+use test;
+
+select * from DS1
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.05.query.sqlpp
new file mode 100644
index 0000000..55a2cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.05.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+// DS2 primary index is on id
+use test;
+
+select * from DS2
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.06.query.sqlpp
new file mode 100644
index 0000000..2b1b11d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.06.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that data scan is used
+ * Expected Res : Success
+ */
+
+// DS2 primary index is on id
+use test;
+
+select * from DS2
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.07.query.sqlpp
new file mode 100644
index 0000000..22ba5b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.07.query.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.
+ */
+
+/*
+ * Description  : Testing that data scan is used
+ * Expected Res : Success
+ */
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = ds1.id
+select ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.08.query.sqlpp
new file mode 100644
index 0000000..4b85cf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.08.query.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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS1 is used
+ * Expected Res : Success
+ */
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = meta(ds1).id
+select meta(ds1).id as ds1_meta_id, ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.09.query.sqlpp
new file mode 100644
index 0000000..0c4c744
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.09.query.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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS2 is used
+ * Expected Res : Success
+ */
+
+// DS2 primary index is on id. DS1 primary index is on meta().id
+use test;
+
+from DS1 as ds1 join DS2 as ds2 on ds1.id /*+ indexnl */ = ds2.id
+select ds1, ds2
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.10.ddl.sqlpp
new file mode 100644
index 0000000..204dfd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.10.ddl.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;
+
+create index id_sec_idx on DS1(id);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.11.query.sqlpp
new file mode 100644
index 0000000..bc12e9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.11.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that secondary index is used
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id & now has a secondary index on id
+use test;
+
+select * from DS1
+where id = "2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.12.query.sqlpp
new file mode 100644
index 0000000..e14e8ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.12.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing that primary index is used
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id & now has a secondary index on id
+use test;
+
+select * from DS1
+where meta().id = "5";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.13.query.sqlpp
new file mode 100644
index 0000000..07f96c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.13.query.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.
+ */
+
+/*
+ * Description  : Testing that secondary index of DS1 is used
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id & now has a secondary index on id. DS2 primary index is on id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = ds1.id
+select ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.14.query.sqlpp
new file mode 100644
index 0000000..f1e3c7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.14.query.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.
+ */
+
+/*
+ * Description  : Testing that primary index of DS1 is used
+ * Expected Res : Success
+ */
+
+// DS1 primary index is on meta().id & now has a secondary index on id. DS2 primary index is on id
+use test;
+
+from DS2 as ds2 join DS1 as ds1 on ds2.id /*+ indexnl */ = meta(ds1).id
+select meta(ds1).id as ds1_meta_id, ds2, ds1
+order by ds2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.15.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.15.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/dataset-with-meta/dataset-with-meta.15.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/meta/meta_in_with_clause/meta_in_with_clause.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.1.ddl.sqlpp
new file mode 100644
index 0000000..e722f40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.1.ddl.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+ /*
+  * Description: tests that meta() used in WITH clause is resolved correctly and produces a correct result
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:int,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:int
+};
+
+create dataset DS(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+create feed DsStream with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header" : "false"
+};
+
+create feed DsStream2 with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "0",
+ "header" : "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.2.update.sqlpp
new file mode 100644
index 0000000..7da99cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 `wait-for-completion-feed` "true";
+connect feed DsStream to dataset DS;
+start feed DsStream;
+
+connect feed DsStream2 to dataset DS2;
+start feed DsStream2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.3.query.sqlpp
new file mode 100644
index 0000000..cfcd79e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+with raw_data as (
+  select meta(a).id as aid, age as age, hobby as hobby
+  from DS a
+  where hobby in ["tennis", "reading","swimming", "hiking", "basketball", "soccer"] )
+
+select t2.raw_data.age, t2.raw_data.aid, array_count(t1) total, t2.raw_data.hobby
+from (select value g from raw_data group by series group as g) as t1 unnest t1 as t2
+order by aid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.4.query.sqlpp
new file mode 100644
index 0000000..0847ae2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+with raw_data as (
+  select id as aid, age as age, hobby as hobby
+  from DS a
+  where hobby in ["tennis", "reading","swimming", "hiking", "basketball", "soccer"] )
+
+select t2.raw_data.age, t2.raw_data.aid, array_count(t1) total, t2.raw_data.hobby
+from (select value g from raw_data group by series group as g) as t1 unnest t1 as t2
+order by aid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.5.query.sqlpp
new file mode 100644
index 0000000..3fcc2fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+with raw_data as (
+  select meta(a).id as aid, age as age, hobby as hobby
+  from DS2 a
+  where hobby in ["tennis", "reading","swimming", "hiking", "basketball", "soccer"] )
+
+select t2.raw_data.age, t2.raw_data.aid, array_count(t1) total, t2.raw_data.hobby
+from (select value g from raw_data group by series group as g) as t1 unnest t1 as t2
+order by aid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.6.query.sqlpp
new file mode 100644
index 0000000..88f7c27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+with raw_data as (
+  select id as aid, age as age, hobby as hobby
+  from DS2 a
+  where hobby in ["tennis", "reading","swimming", "hiking", "basketball", "soccer"] )
+
+select t2.raw_data.age, t2.raw_data.aid, array_count(t1) total, t2.raw_data.hobby
+from (select value g from raw_data group by series group as g) as t1 unnest t1 as t2
+order by aid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.7.ddl.sqlpp
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.7.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;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.1.ddl.sqlpp
new file mode 100644
index 0000000..4297639
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.1.ddl.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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 DataType as open {
+id:int,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:int
+};
+
+create dataset DS(DataType) with meta(MetaType) primary key meta().id;
+create dataset DS2(DataType) with meta(MetaType) primary key id;
+
+create feed DsStream with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header" : "false"
+};
+
+create feed DsStream2 with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "0",
+ "header" : "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.2.update.sqlpp
new file mode 100644
index 0000000..7da99cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 `wait-for-completion-feed` "true";
+connect feed DsStream to dataset DS;
+start feed DsStream;
+
+connect feed DsStream2 to dataset DS2;
+start feed DsStream2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.query.sqlpp
new file mode 100644
index 0000000..39efedb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description: tests resolving primary key correctly
+ */
+
+use test;
+
+from DS
+select meta().id as i
+order by i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.query.sqlpp
new file mode 100644
index 0000000..2b73666
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description: tests that id isn't confused for primary key
+ */
+
+use test;
+
+from DS
+select id as i
+order by i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.query.sqlpp
new file mode 100644
index 0000000..7346025
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description: tests that meta().id isn't confused for primary key
+ */
+
+use test;
+
+from DS2
+select meta().id as i
+order by i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.query.sqlpp
new file mode 100644
index 0000000..40f14ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description: tests resolving primary key correctly
+ */
+
+use test;
+
+from DS2
+select id as i
+order by i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.7.ddl.sqlpp
new file mode 100644
index 0000000..40f14ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.7.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description: tests resolving primary key correctly
+ */
+
+use test;
+
+from DS2
+select id as i
+order by i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.03.adm
new file mode 100644
index 0000000..708eef0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.03.adm
@@ -0,0 +1 @@
+{ "DS1": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.04.adm
new file mode 100644
index 0000000..2c1129d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.04.adm
@@ -0,0 +1 @@
+{ "DS1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.05.adm
new file mode 100644
index 0000000..fd1e899
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.05.adm
@@ -0,0 +1 @@
+{ "DS2": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.06.adm
new file mode 100644
index 0000000..227dd34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.06.adm
@@ -0,0 +1 @@
+{ "DS2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.07.adm
new file mode 100644
index 0000000..43ef810
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.07.adm
@@ -0,0 +1,9 @@
+{ "ds2": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" }, "ds1": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" } }
+{ "ds2": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "ds1": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
+{ "ds2": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" }, "ds1": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" } }
+{ "ds2": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" }, "ds1": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" } }
+{ "ds2": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "ds1": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" } }
+{ "ds2": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "ds1": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" } }
+{ "ds2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "ds1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
+{ "ds2": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "ds1": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" } }
+{ "ds2": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "ds1": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.08.adm
new file mode 100644
index 0000000..99fd8f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.08.adm
@@ -0,0 +1,2 @@
+{ "ds1_meta_id": "5", "ds2": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "ds1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
+{ "ds1_meta_id": "7", "ds2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "ds1": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.09.adm
new file mode 100644
index 0000000..eeb11e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.09.adm
@@ -0,0 +1,9 @@
+{ "ds1": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" }, "ds2": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" } }
+{ "ds1": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "ds2": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
+{ "ds1": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" }, "ds2": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" } }
+{ "ds1": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" }, "ds2": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" } }
+{ "ds1": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "ds2": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" } }
+{ "ds1": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "ds2": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" } }
+{ "ds1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "ds2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
+{ "ds1": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "ds2": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" } }
+{ "ds1": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "ds2": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.11.adm
new file mode 100644
index 0000000..708eef0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.11.adm
@@ -0,0 +1 @@
+{ "DS1": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.12.adm
new file mode 100644
index 0000000..2c1129d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.12.adm
@@ -0,0 +1 @@
+{ "DS1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.13.adm
new file mode 100644
index 0000000..43ef810
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.13.adm
@@ -0,0 +1,9 @@
+{ "ds2": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" }, "ds1": { "id": "1", "name": "John Mad", "age": 29, "hobby": "reading" } }
+{ "ds2": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "ds1": { "id": "2", "name": "Scott Scott", "age": 30, "hobby": "hiking" } }
+{ "ds2": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" }, "ds1": { "id": "3", "name": "Dan David", "age": 40, "hobby": "bowling" } }
+{ "ds2": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" }, "ds1": { "id": "4", "name": "Robert Moore", "age": 32, "hobby": "reading" } }
+{ "ds2": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "ds1": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" } }
+{ "ds2": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "ds1": { "id": "6", "name": "Joe Dana", "age": 24, "hobby": "tennis" } }
+{ "ds2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "ds1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
+{ "ds2": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "ds1": { "id": "8", "name": "Mat Steve", "age": 45, "hobby": "tennis" } }
+{ "ds2": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "ds1": { "id": "9", "name": "Sandra Pec", "age": 36, "hobby": "hiking" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.14.adm
new file mode 100644
index 0000000..99fd8f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/dataset-with-meta/dataset-with-meta.14.adm
@@ -0,0 +1,2 @@
+{ "ds1_meta_id": "5", "ds2": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "ds1": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" } }
+{ "ds1_meta_id": "7", "ds2": { "id": "7", "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "ds1": { "id": "5", "name": "Sandy Donald", "age": 35, "hobby": "soccer" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.3.adm
new file mode 100644
index 0000000..b9ed3c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.3.adm
@@ -0,0 +1,8 @@
+{ "age": 24, "aid": 13071782, "total": 8, "hobby": "tennis" }
+{ "age": 28, "aid": 26237702, "total": 8, "hobby": "basketball" }
+{ "age": 45, "aid": 32571888, "total": 8, "hobby": "tennis" }
+{ "age": 35, "aid": 39225791, "total": 8, "hobby": "soccer" }
+{ "age": 32, "aid": 51041435, "total": 8, "hobby": "reading" }
+{ "age": 30, "aid": 52037425, "total": 8, "hobby": "hiking" }
+{ "age": 29, "aid": 76041664, "total": 8, "hobby": "reading" }
+{ "age": 36, "aid": 86897761, "total": 8, "hobby": "hiking" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.4.adm
new file mode 100644
index 0000000..1cc10ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.4.adm
@@ -0,0 +1,8 @@
+{ "age": 29, "aid": 1, "total": 8, "hobby": "reading" }
+{ "age": 30, "aid": 2, "total": 8, "hobby": "hiking" }
+{ "age": 32, "aid": 4, "total": 8, "hobby": "reading" }
+{ "age": 35, "aid": 5, "total": 8, "hobby": "soccer" }
+{ "age": 24, "aid": 6, "total": 8, "hobby": "tennis" }
+{ "age": 28, "aid": 7, "total": 8, "hobby": "basketball" }
+{ "age": 45, "aid": 8, "total": 8, "hobby": "tennis" }
+{ "age": 36, "aid": 9, "total": 8, "hobby": "hiking" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.5.adm
new file mode 100644
index 0000000..b9ed3c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.5.adm
@@ -0,0 +1,8 @@
+{ "age": 24, "aid": 13071782, "total": 8, "hobby": "tennis" }
+{ "age": 28, "aid": 26237702, "total": 8, "hobby": "basketball" }
+{ "age": 45, "aid": 32571888, "total": 8, "hobby": "tennis" }
+{ "age": 35, "aid": 39225791, "total": 8, "hobby": "soccer" }
+{ "age": 32, "aid": 51041435, "total": 8, "hobby": "reading" }
+{ "age": 30, "aid": 52037425, "total": 8, "hobby": "hiking" }
+{ "age": 29, "aid": 76041664, "total": 8, "hobby": "reading" }
+{ "age": 36, "aid": 86897761, "total": 8, "hobby": "hiking" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.6.adm
new file mode 100644
index 0000000..1cc10ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.6.adm
@@ -0,0 +1,8 @@
+{ "age": 29, "aid": 1, "total": 8, "hobby": "reading" }
+{ "age": 30, "aid": 2, "total": 8, "hobby": "hiking" }
+{ "age": 32, "aid": 4, "total": 8, "hobby": "reading" }
+{ "age": 35, "aid": 5, "total": 8, "hobby": "soccer" }
+{ "age": 24, "aid": 6, "total": 8, "hobby": "tennis" }
+{ "age": 28, "aid": 7, "total": 8, "hobby": "basketball" }
+{ "age": 45, "aid": 8, "total": 8, "hobby": "tennis" }
+{ "age": 36, "aid": 9, "total": 8, "hobby": "hiking" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.adm
new file mode 100644
index 0000000..aca68e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.adm
@@ -0,0 +1,9 @@
+{ "i": 13071782 }
+{ "i": 26237702 }
+{ "i": 32571888 }
+{ "i": 39225791 }
+{ "i": 45962603 }
+{ "i": 51041435 }
+{ "i": 52037425 }
+{ "i": 76041664 }
+{ "i": 86897761 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.adm
new file mode 100644
index 0000000..16bce91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.adm
@@ -0,0 +1,9 @@
+{ "i": 1 }
+{ "i": 2 }
+{ "i": 3 }
+{ "i": 4 }
+{ "i": 5 }
+{ "i": 6 }
+{ "i": 7 }
+{ "i": 8 }
+{ "i": 9 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.adm
new file mode 100644
index 0000000..aca68e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.adm
@@ -0,0 +1,9 @@
+{ "i": 13071782 }
+{ "i": 26237702 }
+{ "i": 32571888 }
+{ "i": 39225791 }
+{ "i": 45962603 }
+{ "i": 51041435 }
+{ "i": 52037425 }
+{ "i": 76041664 }
+{ "i": 86897761 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.adm
new file mode 100644
index 0000000..16bce91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.adm
@@ -0,0 +1,9 @@
+{ "i": 1 }
+{ "i": 2 }
+{ "i": 3 }
+{ "i": 4 }
+{ "i": 5 }
+{ "i": 6 }
+{ "i": 7 }
+{ "i": 8 }
+{ "i": 9 }
\ 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 4fcf9f8..18c5f88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3536,6 +3536,11 @@
         <output-dir compare="Text">disjunctive-predicate-1</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="dataset-with-meta">
+        <output-dir compare="Text">dataset-with-meta</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="inverted-index-join">
     <test-case FilePath="inverted-index-join">
@@ -9052,6 +9057,18 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="meta">
+    <test-case FilePath="meta">
+      <compilation-unit name="meta_in_with_clause">
+        <output-dir compare="Text">meta_in_with_clause</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="resolving_pk_with_meta">
+        <output-dir compare="Text">resolving_pk_with_meta</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="big-object">
     <test-case FilePath="big-object">
       <compilation-unit name="big_object_sort">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 28c612f..c0b36ea 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -41,6 +41,7 @@ import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -173,17 +174,53 @@ public class DatasetUtil {
         return btreeFields;
     }
 
-    public static int getPositionOfPartitioningKeyField(Dataset dataset, List<String> fieldExpr) {
+    /**
+     * Returns the primary key source indicators of the {@code dataset} or {@code null} if the dataset does not have
+     * primary key source indicators (e.g. external datasets)
+     */
+    public static List<Integer> getKeySourceIndicators(Dataset dataset) {
+        IDatasetDetails datasetDetails = dataset.getDatasetDetails();
+        if (datasetDetails.getDatasetType() == DatasetType.INTERNAL) {
+            return ((InternalDatasetDetails) datasetDetails).getKeySourceIndicator();
+        }
+        return null;
+    }
+
+    public static int getPositionOfPartitioningKeyField(Dataset dataset, List<String> fieldExpr,
+            boolean fieldFromMeta) {
+        List<Integer> keySourceIndicator = null;
+        IDatasetDetails datasetDetails = dataset.getDatasetDetails();
+        if (datasetDetails.getDatasetType() == DatasetType.INTERNAL) {
+            keySourceIndicator = ((InternalDatasetDetails) datasetDetails).getKeySourceIndicator();
+        }
         List<List<String>> partitioningKeys = dataset.getPrimaryKeys();
         for (int i = 0; i < partitioningKeys.size(); i++) {
             List<String> partitioningKey = partitioningKeys.get(i);
-            if (partitioningKey.equals(fieldExpr)) {
+            if (partitioningKey.equals(fieldExpr) && keySourceMatches(keySourceIndicator, i, fieldFromMeta)) {
                 return i;
             }
         }
         return -1;
     }
 
+    /**
+     * Once it's determined that a field name is a key (by just comparing the names), this method checks whether the
+     * field is actually a key by making sure the field is coming from the right record (data record or meta record),
+     * e.g. if the field name happens to be equal to the key name but the field is coming from the data record while
+     * the key is coming from the meta record.
+     * @param keySourceIndicator indicates where the key is coming from, 1 from meta record, 0 from data record
+     * @param keyIndex the key index we're checking the field against
+     * @param fieldFromMeta whether the field is coming from the meta record or the data record
+     * @return true if the key source matches the field source. Otherwise, false.
+     */
+    private static boolean keySourceMatches(List<Integer> keySourceIndicator, int keyIndex, boolean fieldFromMeta) {
+        if (keySourceIndicator != null) {
+            return (fieldFromMeta && keySourceIndicator.get(keyIndex) == 1)
+                    || (!fieldFromMeta && keySourceIndicator.get(keyIndex) == 0);
+        }
+        return true;
+    }
+
     public static Pair<ILSMMergePolicyFactory, Map<String, String>> getMergePolicyFactory(Dataset dataset,
             MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
         String policyName = dataset.getCompactionPolicy();