You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2018/11/07 01:30:33 UTC

[2/2] asterixdb git commit: [NO ISSUE][COMP] fix resolving PK field accesses & rewriting of meta()

[NO ISSUE][COMP] fix resolving PK field accesses & rewriting of meta()

- user model changes: no
- storage format changes: no
- interface changes: no

details:
This patch 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: I2d38cbfcc4a0eee51abf753a7d958dafa666aabf
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3018
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/8076fe97
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/8076fe97
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/8076fe97

Branch: refs/heads/master
Commit: 8076fe97d16d9b2774718739dc39b4961c3b8ea2
Parents: a57e154
Author: Ali Alsuliman <al...@gmail.com>
Authored: Mon Nov 5 14:56:09 2018 -0800
Committer: Ali Alsuliman <al...@gmail.com>
Committed: Tue Nov 6 17:29:05 2018 -0800

----------------------------------------------------------------------
 .../asterix/optimizer/base/RuleCollections.java |   4 +-
 .../rules/MetaFunctionToMetaVariableRule.java   |  39 +++--
 .../optimizer/rules/PushFieldAccessRule.java    | 162 +++++++++++--------
 asterixdb/asterix-app/data/csv/people.csv       |   9 ++
 .../field_access_with_data_and_meta_1.sqlpp     |  41 +++++
 .../field_access_with_data_and_meta_2.sqlpp     |  41 +++++
 .../field_access_with_data_and_meta_3.sqlpp     |  41 +++++
 .../field_access_with_data_and_meta_4.sqlpp     |  41 +++++
 .../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 ++
 .../results/meta/with_clause_meta.plan          |  44 +++++
 .../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          |  20 +++
 .../meta_in_with_clause.3.adm                   |   8 +
 .../meta_in_with_clause.4.adm                   |   8 +
 .../meta_in_with_clause.5.adm                   |   8 +
 .../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 ++
 .../src/test/resources/runtimets/testsuite.xml  |   2 +-
 .../resources/runtimets/testsuite_sqlpp.xml     |  12 ++
 .../asterix/metadata/utils/DatasetUtil.java     |  36 ++++-
 39 files changed, 1004 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
----------------------------------------------------------------------
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 e7fb579..ece78f8 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
@@ -240,7 +240,8 @@ 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());
-
+        // once meta() is replaced before logical plan generation, this can be removed
+        condPushDownAndJoinInference.add(new MetaFunctionToMetaVariableRule());
         return condPushDownAndJoinInference;
     }
 
@@ -249,6 +250,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());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
----------------------------------------------------------------------
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..a6533cf 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,14 +45,15 @@ 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;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
- * This rule rewrites all meta() function calls in a query plan
- * to proper variable references.
+ * This rule rewrites all meta() and meta-key() function calls in a query plan to proper variable references.
  */
 public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
     // The rule can only apply once.
@@ -74,10 +76,10 @@ public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
         ILogicalOperator op = opRef.getValue();
 
         // Reaches NTS or ETS.
-        if (op.getInputs().size() == 0) {
+        if (op.getInputs().isEmpty()) {
             return NoOpExpressionReferenceTransform.INSTANCE;
         }
-        // Datascan returns an useful transform if the meta part presents in the dataset.
+        // Datascan returns a useful transform if the meta part is present in the dataset.
         if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator scanOp = (DataSourceScanOperator) op;
             ILogicalExpressionReferenceTransformWithCondition inputTransfomer = visit(op.getInputs().get(0));
@@ -137,7 +139,7 @@ public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
             }
         }
         ILogicalExpressionReferenceTransformWithCondition currentTransformer = null;
-        if (transformers.size() == 0) {
+        if (transformers.isEmpty()) {
             currentTransformer = NoOpExpressionReferenceTransform.INSTANCE;
         } else if (transformers.size() == 1) {
             currentTransformer = transformers.get(0);
@@ -148,6 +150,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;
     }
@@ -167,7 +178,7 @@ class NoOpExpressionReferenceTransform implements ILogicalExpressionReferenceTra
     }
 
     @Override
-    public boolean transform(Mutable<ILogicalExpression> expression) throws AlgebricksException {
+    public boolean transform(Mutable<ILogicalExpression> expression) {
         return false;
     }
 
@@ -197,13 +208,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 +227,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());
@@ -231,8 +243,7 @@ class LogicalExpressionReferenceTransform implements ILogicalExpressionReference
         // The user query provides zero parameter for the meta function.
         if (variableRequired) {
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
-                    "Cannot resolve to ambiguity on the meta function call --"
-                            + " there are more than one dataset choices!");
+                    "Cannot resolve ambiguous meta function call. There are more than one dataset choice!");
         }
         VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
         metaVarRef.setSourceLocation(expr.getSourceLocation());
@@ -250,7 +261,7 @@ class CompositeExpressionReferenceTransform implements ILogicalExpressionReferen
 
     @Override
     public boolean transform(Mutable<ILogicalExpression> expression) throws AlgebricksException {
-        // Tries transfomations one by one.
+        // tries transformations one by one.
         for (ILogicalExpressionReferenceTransform transformer : transformers) {
             if (transformer.transform(expression)) {
                 return true;
@@ -323,7 +334,7 @@ class MetaKeyExpressionReferenceTransform implements ILogicalExpressionReference
     }
 
     @Override
-    public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
+    public boolean transform(Mutable<ILogicalExpression> exprRef) {
         ILogicalExpression expr = exprRef.getValue();
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
----------------------------------------------------------------------
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..e6f5d91 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
@@ -64,6 +64,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceSc
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -94,7 +95,7 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         } else {
             return false;
         }
-        return propagateFieldAccessRec(opRef, context, finalAnnot);
+        return pushDownFieldAccessRec(opRef, context, finalAnnot);
     }
 
     private boolean isAccessToIndexedField(AssignOperator assign, IOptimizationContext context)
@@ -173,36 +174,37 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         return e1.equals(e2);
     }
 
-    private boolean propagateFieldAccessRec(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+    private boolean pushDownFieldAccessRec(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
             String finalAnnot) throws AlgebricksException {
-        AssignOperator access = (AssignOperator) opRef.getValue();
-        Mutable<ILogicalOperator> opRef2 = access.getInputs().get(0);
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
-        // If it's not an indexed field, it is pushed so that scan can be
-        // rewritten into index search.
-        if (op2.getOperatorTag() == LogicalOperatorTag.PROJECT || context.checkAndAddToAlreadyCompared(access, op2)
-                && !(op2.getOperatorTag() == LogicalOperatorTag.SELECT && isAccessToIndexedField(access, context))) {
+        AssignOperator assignOp = (AssignOperator) opRef.getValue();
+        Mutable<ILogicalOperator> opRef2 = assignOp.getInputs().get(0);
+        AbstractLogicalOperator inputOp = (AbstractLogicalOperator) opRef2.getValue();
+        // If it's not an indexed field, it is pushed so that scan can be rewritten into index search.
+        if (inputOp.getOperatorTag() == LogicalOperatorTag.PROJECT
+                || context.checkAndAddToAlreadyCompared(assignOp, inputOp)
+                        && !(inputOp.getOperatorTag() == LogicalOperatorTag.SELECT
+                                && isAccessToIndexedField(assignOp, context))) {
             return false;
         }
-        Object annotation = op2.getAnnotations().get(OperatorPropertiesUtil.MOVABLE);
+        Object annotation = inputOp.getAnnotations().get(OperatorPropertiesUtil.MOVABLE);
         if (annotation != null && !((Boolean) annotation)) {
             return false;
         }
-        if (tryingToPushThroughSelectionWithSameDataSource(access, op2)) {
+        if (tryingToPushThroughSelectionWithSameDataSource(assignOp, inputOp)) {
             return false;
         }
-        if (testAndModifyRedundantOp(access, op2)) {
-            propagateFieldAccessRec(opRef2, context, finalAnnot);
+        if (testAndModifyRedundantOp(assignOp, inputOp)) {
+            pushDownFieldAccessRec(opRef2, context, finalAnnot);
             return true;
         }
         List<LogicalVariable> usedInAccess = new LinkedList<>();
-        VariableUtilities.getUsedVariables(access, usedInAccess);
+        VariableUtilities.getUsedVariables(assignOp, usedInAccess);
 
         List<LogicalVariable> produced2 = new LinkedList<>();
-        if (op2.getOperatorTag() == LogicalOperatorTag.GROUP) {
-            VariableUtilities.getLiveVariables(op2, produced2);
+        if (inputOp.getOperatorTag() == LogicalOperatorTag.GROUP) {
+            VariableUtilities.getLiveVariables(inputOp, produced2);
         } else {
-            VariableUtilities.getProducedVariables(op2, produced2);
+            VariableUtilities.getProducedVariables(inputOp, produced2);
         }
         boolean pushItDown = false;
         List<LogicalVariable> inter = new ArrayList<>(usedInAccess);
@@ -212,8 +214,8 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         inter.retainAll(produced2);
         if (inter.isEmpty()) {
             pushItDown = true;
-        } else if (op2.getOperatorTag() == LogicalOperatorTag.GROUP) {
-            GroupByOperator g = (GroupByOperator) op2;
+        } else if (inputOp.getOperatorTag() == LogicalOperatorTag.GROUP) {
+            GroupByOperator g = (GroupByOperator) inputOp;
             List<Pair<LogicalVariable, LogicalVariable>> varMappings = new ArrayList<>();
             for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : g.getDecorList()) {
                 ILogicalExpression e = p.second.getValue();
@@ -230,67 +232,63 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
                 boolean changed = false;
                 for (Pair<LogicalVariable, LogicalVariable> m : varMappings) {
                     LogicalVariable v2 = context.newVar();
-                    LogicalVariable oldVar = access.getVariables().get(0);
+                    LogicalVariable oldVar = assignOp.getVariables().get(0);
                     VariableReferenceExpression v2Ref = new VariableReferenceExpression(v2);
                     v2Ref.setSourceLocation(g.getSourceLocation());
                     g.getDecorList().add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(oldVar,
                             new MutableObject<ILogicalExpression>(v2Ref)));
                     changed = true;
-                    access.getVariables().set(0, v2);
-                    VariableUtilities.substituteVariables(access, m.first, m.second, context);
+                    assignOp.getVariables().set(0, v2);
+                    VariableUtilities.substituteVariables(assignOp, m.first, m.second, context);
                 }
                 if (changed) {
                     context.computeAndSetTypeEnvironmentForOperator(g);
                 }
                 usedInAccess.clear();
-                VariableUtilities.getUsedVariables(access, usedInAccess);
+                VariableUtilities.getUsedVariables(assignOp, usedInAccess);
                 pushItDown = true;
             }
         }
         if (pushItDown) {
-            if (op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            if (inputOp.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
                 Mutable<ILogicalOperator> childOfSubplan =
-                        ((NestedTupleSourceOperator) op2).getDataSourceReference().getValue().getInputs().get(0);
-                pushAccessDown(opRef, op2, childOfSubplan, context, finalAnnot);
+                        ((NestedTupleSourceOperator) inputOp).getDataSourceReference().getValue().getInputs().get(0);
+                pushAccessDown(opRef, inputOp, childOfSubplan, context, finalAnnot);
                 return true;
             }
-            if (op2.getInputs().size() == 1 && !op2.hasNestedPlans()) {
-                pushAccessDown(opRef, op2, op2.getInputs().get(0), context, finalAnnot);
+            if (inputOp.getInputs().size() == 1 && !inputOp.hasNestedPlans()) {
+                pushAccessDown(opRef, inputOp, inputOp.getInputs().get(0), context, finalAnnot);
                 return true;
             } else {
-                for (Mutable<ILogicalOperator> inp : op2.getInputs()) {
+                for (Mutable<ILogicalOperator> inp : inputOp.getInputs()) {
                     HashSet<LogicalVariable> v2 = new HashSet<>();
                     VariableUtilities.getLiveVariables(inp.getValue(), v2);
                     if (v2.containsAll(usedInAccess)) {
-                        pushAccessDown(opRef, op2, inp, context, finalAnnot);
+                        pushAccessDown(opRef, inputOp, inp, context, finalAnnot);
                         return true;
                     }
                 }
             }
-            if (op2.hasNestedPlans()) {
-                AbstractOperatorWithNestedPlans nestedOp = (AbstractOperatorWithNestedPlans) op2;
+            if (inputOp.hasNestedPlans()) {
+                AbstractOperatorWithNestedPlans nestedOp = (AbstractOperatorWithNestedPlans) inputOp;
                 for (ILogicalPlan plan : nestedOp.getNestedPlans()) {
                     for (Mutable<ILogicalOperator> root : plan.getRoots()) {
                         HashSet<LogicalVariable> v2 = new HashSet<>();
                         VariableUtilities.getLiveVariables(root.getValue(), v2);
                         if (v2.containsAll(usedInAccess)) {
-                            pushAccessDown(opRef, op2, root, context, finalAnnot);
+                            pushAccessDown(opRef, inputOp, root, context, finalAnnot);
                             return true;
                         }
                     }
                 }
             }
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, access.getSourceLocation(),
-                    "Field access " + access.getExpressions().get(0).getValue()
-                            + " does not correspond to any input of operator " + op2);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, assignOp.getSourceLocation(),
+                    "Field access " + assignOp.getExpressions().get(0).getValue()
+                            + " does not correspond to any input of operator " + inputOp);
         } else {
-            // Check if the accessed field is not one of the partitioning key
-            // 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);
-
+            // check if the accessed field is one of the partitioning key fields. If yes, we can equate the 2 variables
+            if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator scan = (DataSourceScanOperator) inputOp;
                 IDataSource<DataSourceId> dataSource = (IDataSource<DataSourceId>) scan.getDataSource();
                 byte dsType = ((DataSource) dataSource).getDatasourceType();
                 if (dsType != DataSource.Type.INTERNAL_DATASET && dsType != DataSource.Type.EXTERNAL_DATASET) {
@@ -304,43 +302,71 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
                             asid.getDatasourceName(), asid.getDataverseName());
                 }
                 if (dataset.getDatasetType() != DatasetType.INTERNAL) {
-                    setAsFinal(access, context, finalAnnot);
+                    setAsFinal(assignOp, context, finalAnnot);
                     return false;
                 }
 
-                String tName = dataset.getItemTypeName();
-                IAType t = mp.findType(dataset.getItemTypeDataverseName(), tName);
-                if (t.getTypeTag() != ATypeTag.OBJECT) {
+                List<LogicalVariable> allVars = scan.getVariables();
+                LogicalVariable dataRecVarInScan = ((DataSource) dataSource).getDataRecordVariable(allVars);
+                LogicalVariable metaRecVarInScan = ((DataSource) dataSource).getMetaVariable(allVars);
+
+                // data part
+                String dataTypeName = dataset.getItemTypeName();
+                IAType dataType = mp.findType(dataset.getItemTypeDataverseName(), dataTypeName);
+                if (dataType.getTypeTag() != ATypeTag.OBJECT) {
                     return false;
                 }
-                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;
+                ARecordType dataRecType = (ARecordType) dataType;
+                Pair<ILogicalExpression, List<String>> fieldPathAndVar = getFieldExpression(assignOp, dataRecType);
+                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(assignOp, (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(assignOp, context, finalAnnot);
+                        return false;
                     }
+                    LogicalVariable keyVar = scan.getVariables().get(p);
+                    VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                    keyVarRef.setSourceLocation(targetRecVar.getSourceLocation());
+                    assignOp.getExpressions().get(0).setValue(keyVarRef);
+                    return true;
                 }
             }
-            setAsFinal(access, context, finalAnnot);
+            setAsFinal(assignOp, context, finalAnnot);
             return false;
         }
     }
 
-    private Pair<ILogicalExpression, List<String>> getFieldExpression(AssignOperator access, ARecordType rt)
-            throws AlgebricksException {
+    /**
+     * @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) {
         LinkedList<String> fieldPath = new LinkedList<>();
         ILogicalExpression e0 = access.getExpressions().get(0).getValue();
         while (AnalysisUtil.isAccessToFieldRecord(e0)) {
@@ -388,7 +414,7 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         }
     }
 
-    // indirect recursivity with propagateFieldAccessRec
+    // indirect recursivity with pushDownFieldAccessRec
     private void pushAccessDown(Mutable<ILogicalOperator> fldAccessOpRef, ILogicalOperator op2,
             Mutable<ILogicalOperator> inputOfOp2, IOptimizationContext context, String finalAnnot)
             throws AlgebricksException {
@@ -401,7 +427,7 @@ public class PushFieldAccessRule implements IAlgebraicRewriteRule {
         // typing
         context.computeAndSetTypeEnvironmentForOperator(fieldAccessOp);
         context.computeAndSetTypeEnvironmentForOperator(op2);
-        propagateFieldAccessRec(inputOfOp2, context, finalAnnot);
+        pushDownFieldAccessRec(inputOfOp2, context, finalAnnot);
     }
 
     private ILogicalExpression getFirstExpr(AssignOperator assign) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/data/csv/people.csv
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_1.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_2.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_3.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/field_access_with_data_and_meta_4.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/with_clause_meta.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_1.plan
----------------------------------------------------------------------
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..4bef88b
--- /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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_2.plan
----------------------------------------------------------------------
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..697008a
--- /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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_3.plan
----------------------------------------------------------------------
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..697008a
--- /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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/field_access_with_data_and_meta_4.plan
----------------------------------------------------------------------
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..4bef88b
--- /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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/with_clause_meta.plan
----------------------------------------------------------------------
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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.1.ddl.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.2.update.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.3.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.4.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.5.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.6.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_in_with_clause/meta_in_with_clause.7.ddl.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.1.ddl.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.2.update.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.query.sqlpp
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/resolving_pk_with_meta/resolving_pk_with_meta.7.ddl.sqlpp
----------------------------------------------------------------------
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..f12a2b7
--- /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,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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.3.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.4.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.5.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_in_with_clause/meta_in_with_clause.6.adm
----------------------------------------------------------------------
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..7bffc6d
--- /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" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.3.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.4.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.5.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/resolving_pk_with_meta/resolving_pk_with_meta.6.adm
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 43c6782..5c7e86d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -783,7 +783,7 @@
     <test-case FilePath="meta">
       <compilation-unit name="query_dataset_with_meta_failure">
         <output-dir compare="Text">query_dataset_with_meta_failure</output-dir>
-        <expected-error>Cannot resolve to ambiguity on the meta function call -- there are more than one dataset choices!</expected-error>
+        <expected-error>Cannot resolve ambiguous meta function call. There are more than one dataset choice!</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/8076fe97/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
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 619d12c..ed864cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9985,6 +9985,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">