You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/07/21 00:42:38 UTC

asterixdb git commit: ASTERIXDB-1536: supports fully qualified dataset path in SQL++.

Repository: asterixdb
Updated Branches:
  refs/heads/master dc286ff91 -> 74fec83d9


ASTERIXDB-1536: supports fully qualified dataset path in SQL++.

Change-Id: I93c7187b3a363a82dbfa225eb67ab526e04aa2dd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1008
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


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

Branch: refs/heads/master
Commit: 74fec83d91bd658b9f474ab65ea8febb3efb8d78
Parents: dc286ff
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Wed Jul 20 16:10:47 2016 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Jul 20 17:42:20 2016 -0700

----------------------------------------------------------------------
 .../optimizer/rules/ResolveVariableRule.java    | 159 ++++++++++++++-----
 .../parserts/queries_sqlpp/columnalias2.sqlpp   |   2 +-
 .../results_parser_sqlpp/columnalias2.ast       |  14 +-
 .../custord/join_q_08/join_q_08.1.ddl.sqlpp     |  61 +++++++
 .../custord/join_q_08/join_q_08.2.update.sqlpp  |  20 +++
 .../custord/join_q_08/join_q_08.3.query.sqlpp   |  29 ++++
 .../custord/join_q_09/join_q_09.1.ddl.sqlpp     |  61 +++++++
 .../custord/join_q_09/join_q_09.2.query.sqlpp   |  29 ++++
 .../resources/runtimets/testsuite_sqlpp.xml     |  17 +-
 .../visitor/VariableCheckAndRewriteVisitor.java |  86 +++++++---
 10 files changed, 402 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
index 4965197..d407a7b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
@@ -38,6 +38,7 @@ import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -74,7 +75,8 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
         }
         // Populates the latest type information, e.g., resolved path sugars.
         context.computeAndSetTypeEnvironmentForOperator(op);
-        if (op.acceptExpressionTransform(exprRef -> rewriteExpressionReference(op, exprRef, context))) {
+        if (op.acceptExpressionTransform(
+                exprRef -> rewriteExpressionReference(op, exprRef, new Triple<>(false, null, null), null, context))) {
             // Generates the up-to-date type information.
             context.computeAndSetTypeEnvironmentForOperator(op);
             return true;
@@ -84,40 +86,45 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
 
     // Recursively rewrites for an expression within an operator.
     private boolean rewriteExpressionReference(ILogicalOperator op, Mutable<ILogicalExpression> exprRef,
-            IOptimizationContext context) throws AlgebricksException {
+            Triple<Boolean, String, String> fullyQualifiedDatasetPathCandidateFromParent,
+            Mutable<ILogicalExpression> parentFuncRef, IOptimizationContext context) throws AlgebricksException {
         ILogicalExpression expr = exprRef.getValue();
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         boolean changed = false;
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        Triple<Boolean, String, String> fullyQualifiedDatasetPathCandidate =
+                resolveFullyQualifiedPath(funcExpr, context);
         for (Mutable<ILogicalExpression> funcArgRef : funcExpr.getArguments()) {
-            if (rewriteExpressionReference(op, funcArgRef, context)) {
-                context.computeAndSetTypeEnvironmentForOperator(op);
+            if (rewriteExpressionReference(op, funcArgRef, fullyQualifiedDatasetPathCandidate, exprRef, context)) {
                 changed = true;
             }
         }
 
         // Cleans up extra scan-collections if there is.
-        cleanupScanCollectionForDataset(funcExpr);
+        if (changed) {
+            cleanupScanCollectionForDataset(funcExpr);
+        }
 
         // Does the actual resolution.
-        return changed || resolve(op, context, exprRef);
+        return changed || resolve(op, context, exprRef, fullyQualifiedDatasetPathCandidateFromParent, parentFuncRef);
     }
 
     // Resolves a "resolve" function call expression to a fully qualified variable/field-access path or
     // a dataset.
-    private boolean resolve(ILogicalOperator op, IOptimizationContext context, Mutable<ILogicalExpression> exprRef)
-            throws AlgebricksException {
+    private boolean resolve(ILogicalOperator op, IOptimizationContext context, Mutable<ILogicalExpression> exprRef,
+            Triple<Boolean, String, String> fullyQualifiedDatasetPathCandidateFromParent,
+            Mutable<ILogicalExpression> parentFuncRef) throws AlgebricksException {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) exprRef.getValue();
         if (funcExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.RESOLVE) {
             return false;
         }
         ILogicalExpression arg = funcExpr.getArguments().get(0).getValue();
-        String unresolvedVarName = extractVariableName(arg);
-        return resolveInternal(exprRef, hasMatchedDataset(unresolvedVarName, context),
+        String unresolvedVarName = extractConstantString(arg);
+        return resolveInternal(exprRef, hasMatchedDatasetForVariableName(unresolvedVarName, context),
                 findCandidatePaths(op, extractPossibleVariables(funcExpr.getArguments()), unresolvedVarName, context),
-                unresolvedVarName);
+                unresolvedVarName, fullyQualifiedDatasetPathCandidateFromParent, parentFuncRef);
     }
 
     // Extracts all possible variables from the arguments of the "resolve" function.
@@ -134,41 +141,70 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
     // Resolves an undefined name to a dataset or a fully qualified variable/field-access path
     // based on the given information of dataset matches and candidate paths.
     private boolean resolveInternal(Mutable<ILogicalExpression> funcRef, boolean hasMatchedDataset,
-            Collection<Pair<LogicalVariable, List<String>>> varAccessCandidates, String unresolvedVarName)
-            throws AlgebricksException {
+            Collection<Pair<LogicalVariable, List<String>>> varAccessCandidates, String unresolvedVarName,
+            Triple<Boolean, String, String> fullyQualifiedDatasetPathCandidateFromParent,
+            Mutable<ILogicalExpression> parentFuncRef) throws AlgebricksException {
         AbstractFunctionCallExpression func = (AbstractFunctionCallExpression) funcRef.getValue();
         int numVarCandidates = varAccessCandidates.size();
-        if (numVarCandidates > 1 || (numVarCandidates == 1 && hasMatchedDataset)) {
+        boolean hasAmbiguity =
+                hasAmbiguity(hasMatchedDataset, fullyQualifiedDatasetPathCandidateFromParent, numVarCandidates);
+        if (hasAmbiguity) {
+            // More than one possibilities.
             throw new AlgebricksException(
                     "Cannot resolve ambiguous alias (variable) reference for identifier " + unresolvedVarName);
-        } else if (numVarCandidates <= 0) {
-            if (!hasMatchedDataset) {
-                throw new AlgebricksException(
-                        "Undefined alias (variable) reference for identifier " + unresolvedVarName);
-            }
-            // Rewrites the "resolve" function to a "dataset" function.
+        } else if (hasMatchedDataset) {
+            // Rewrites the "resolve" function to a "dataset" function and only keep the dataset name argument.
             func.setFunctionInfo(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.DATASET));
+            Mutable<ILogicalExpression> datasetNameExpression = func.getArguments().get(0);
+            func.getArguments().clear();
+            func.getArguments().add(datasetNameExpression);
+        } else if (fullyQualifiedDatasetPathCandidateFromParent.first) {
+            // Rewrites the parent "field-access" function to a "dataset" function.
+            AbstractFunctionCallExpression parentFunc = (AbstractFunctionCallExpression) parentFuncRef.getValue();
+            parentFunc.setFunctionInfo(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.DATASET));
+            parentFunc.getArguments().clear();
+            parentFunc.getArguments()
+                    .add(new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AString(fullyQualifiedDatasetPathCandidateFromParent.second
+                                    + "." + fullyQualifiedDatasetPathCandidateFromParent.third)))));
+        } else if (numVarCandidates == 1) {
+            resolveAsFieldAccess(funcRef, varAccessCandidates.iterator().next());
         } else {
-            // Rewrites to field-access-by-names.
-            Pair<LogicalVariable, List<String>> varAndPath = varAccessCandidates.iterator().next();
-            LogicalVariable var = varAndPath.first;
-            List<String> path = varAndPath.second;
-            Mutable<ILogicalExpression> firstArgRef = new MutableObject<>(new VariableReferenceExpression(var));
-            ILogicalExpression newFunc = null;
-            for (String fieldName : path) {
-                List<Mutable<ILogicalExpression>> args = new ArrayList<>();
-                args.add(firstArgRef);
-                args.add(new MutableObject<ILogicalExpression>(
-                        new ConstantExpression(new AsterixConstantValue(new AString(fieldName)))));
-                newFunc = new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME), args);
-                firstArgRef = new MutableObject<>(newFunc);
-            }
-            funcRef.setValue(newFunc);
+            // Cannot find any resolution.
+            throw new AlgebricksException("Undefined alias (variable) reference for identifier " + unresolvedVarName);
         }
         return true;
     }
 
+    // Check whether it is possible to have multiple resolutions for a "resolve" function.
+    private boolean hasAmbiguity(boolean hasMatchedDataset,
+            Triple<Boolean, String, String> fullyQualifiedDatasetPathCandidateFromParent, int numVarCandidates) {
+        boolean hasAmbiguity = numVarCandidates > 1 || (numVarCandidates == 1 && hasMatchedDataset);
+        hasAmbiguity = hasAmbiguity || (numVarCandidates == 1 && fullyQualifiedDatasetPathCandidateFromParent.first);
+        hasAmbiguity = hasAmbiguity || (hasMatchedDataset && fullyQualifiedDatasetPathCandidateFromParent.first);
+        return hasAmbiguity;
+    }
+
+    // Resolves a "resolve" function call as a field access.
+    private void resolveAsFieldAccess(Mutable<ILogicalExpression> funcRef,
+            Pair<LogicalVariable, List<String>> varAndPath) {
+        // Rewrites to field-access-by-names.
+        LogicalVariable var = varAndPath.first;
+        List<String> path = varAndPath.second;
+        Mutable<ILogicalExpression> firstArgRef = new MutableObject<>(new VariableReferenceExpression(var));
+        ILogicalExpression newFunc = null;
+        for (String fieldName : path) {
+            List<Mutable<ILogicalExpression>> args = new ArrayList<>();
+            args.add(firstArgRef);
+            args.add(new MutableObject<ILogicalExpression>(
+                    new ConstantExpression(new AsterixConstantValue(new AString(fieldName)))));
+            newFunc = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME), args);
+            firstArgRef = new MutableObject<>(newFunc);
+        }
+        funcRef.setValue(newFunc);
+    }
+
     // Finds all candidate fully qualified variable/field-access paths.
     private Set<Pair<LogicalVariable, List<String>>> findCandidatePaths(ILogicalOperator op,
             Collection<LogicalVariable> inputLiveVars, String unresolvedVarName, IOptimizationContext context)
@@ -218,11 +254,51 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
         return varAccessCandidates;
     }
 
+    // Try to resolve the expression like resolve("x").foo as x.foo.
+    private Triple<Boolean, String, String> resolveFullyQualifiedPath(AbstractFunctionCallExpression funcExpr,
+            IOptimizationContext context) throws AlgebricksException {
+        if (!funcExpr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+            return new Triple<>(false, null, null);
+        }
+        List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+        ILogicalExpression firstExpr = args.get(0).getValue();
+        ILogicalExpression secondExpr = args.get(1).getValue();
+        if (firstExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return new Triple<>(false, null, null);
+        }
+        if (secondExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return new Triple<>(false, null, null);
+        }
+        AbstractFunctionCallExpression firstFuncExpr = (AbstractFunctionCallExpression) firstExpr;
+        if (!firstFuncExpr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.RESOLVE)) {
+            return new Triple<>(false, null, null);
+        }
+        ILogicalExpression dataverseNameExpr = firstFuncExpr.getArguments().get(0).getValue();
+        String dataverseName = extractConstantString(dataverseNameExpr);
+        String datasetName = extractConstantString(secondExpr);
+        return new Triple<>(hasMatchedDataverseDataset(dataverseName, datasetName, context), dataverseName,
+                datasetName);
+    }
+
+    // Checks whether the dataverse name and dataset name matche a dataset.
+    private boolean hasMatchedDataverseDataset(String dataverseName, String datasetName, IOptimizationContext context)
+            throws AlgebricksException {
+        AqlMetadataProvider mdp = (AqlMetadataProvider) context.getMetadataProvider();
+        if (mdp.findDataset(dataverseName, datasetName) != null) {
+            return true;
+        }
+        return false;
+    }
+
     // Checks whether the name matches a dataset.
-    private boolean hasMatchedDataset(String name, IOptimizationContext context) throws AlgebricksException {
+    private boolean hasMatchedDatasetForVariableName(String varName, IOptimizationContext context)
+            throws AlgebricksException {
         AqlMetadataProvider mdp = (AqlMetadataProvider) context.getMetadataProvider();
-        if (name.contains(".")) {
-            String[] path = name.split("\\.");
+        if (mdp.findDataset(mdp.getDefaultDataverseName(), varName) != null) {
+            return true;
+        }
+        if (varName.contains(".")) {
+            String[] path = varName.split("\\.");
             if (path.length != 2) {
                 return false;
             }
@@ -230,9 +306,6 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
                 return true;
             }
         }
-        if (mdp.findDataset(mdp.getDefaultDataverseName(), name) != null) {
-            return true;
-        }
         return false;
     }
 
@@ -267,7 +340,7 @@ public class ResolveVariableRule implements IAlgebraicRewriteRule {
     }
 
     // Extracts the name of an undefined variable.
-    private String extractVariableName(ILogicalExpression arg) throws AlgebricksException {
+    private String extractConstantString(ILogicalExpression arg) throws AlgebricksException {
         if (arg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
             throw new AlgebricksException("The argument is expected to be a constant value.");
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/columnalias2.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/columnalias2.sqlpp b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/columnalias2.sqlpp
index 8eeea85..f74cf47 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/columnalias2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/columnalias2.sqlpp
@@ -21,4 +21,4 @@ SELECT SQRT(t.a*t.b) AS root FROM tbl_name root
 GROUP BY root.id
 WITH u AS root.time
 HAVING root.orders > 0
-ORDER BY u; 
\ No newline at end of file
+ORDER BY u;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
index 0b7ccd2..89e3b4b 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
@@ -2,18 +2,12 @@ Query:
 SELECT [
 FunctionCall null.SQRT@1[
   OperatorExpr [
-    FieldAccessor [
-      FunctionCall Metadata.dataset@1[
-        LiteralExpr [STRING] [t]
-      ]
-      Field=a
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [t.a]
     ]
     *
-    FieldAccessor [
-      FunctionCall Metadata.dataset@1[
-        LiteralExpr [STRING] [t]
-      ]
-      Field=b
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [t.b]
     ]
   ]
 ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.1.ddl.sqlpp
new file mode 100644
index 0000000..193d6b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.1.ddl.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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  database test if exists;
+create  database test;
+
+use test;
+
+
+create type test.AddressType as
+{
+  number : int64,
+  street : string,
+  city : string
+}
+
+create type test.CustomerType as
+ closed {
+  cid : int64,
+  name : string,
+  cashBack : int64,
+  age : int64?,
+  address : AddressType?,
+  lastorder : {
+      oid : int64,
+      total : float
+  }
+
+}
+
+create type test.OrderType as
+{
+  oid : int64,
+  cid : int64,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float,
+  items : [int64]
+}
+
+create external table Customers(CustomerType) using localfs((`path`=`asterix_nc1://data/nontagged/customerData.json`),(`format`=`adm`));
+
+create external table Orders(OrderType) using localfs((`path`=`asterix_nc1://data/nontagged/orderData.json`),(`format`=`adm`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.2.update.sqlpp
new file mode 100644
index 0000000..6c98c1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.2.update.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.
+ */
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.3.query.sqlpp
new file mode 100644
index 0000000..13fd8f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_08/join_q_08.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.
+ */
+
+/** This query is to test fully qualified path for datasets. */
+
+SELECT c.name AS cust_name,
+       c.age AS cust_age,
+       o.total AS order_total,
+       [o.oid,o.cid] AS orderList
+FROM test.Customers c, test.Orders o
+WHERE c.cid = o.cid
+ORDER BY c.name,o.total
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.1.ddl.sqlpp
new file mode 100644
index 0000000..193d6b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.1.ddl.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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  database test if exists;
+create  database test;
+
+use test;
+
+
+create type test.AddressType as
+{
+  number : int64,
+  street : string,
+  city : string
+}
+
+create type test.CustomerType as
+ closed {
+  cid : int64,
+  name : string,
+  cashBack : int64,
+  age : int64?,
+  address : AddressType?,
+  lastorder : {
+      oid : int64,
+      total : float
+  }
+
+}
+
+create type test.OrderType as
+{
+  oid : int64,
+  cid : int64,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float,
+  items : [int64]
+}
+
+create external table Customers(CustomerType) using localfs((`path`=`asterix_nc1://data/nontagged/customerData.json`),(`format`=`adm`));
+
+create external table Orders(OrderType) using localfs((`path`=`asterix_nc1://data/nontagged/orderData.json`),(`format`=`adm`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.2.query.sqlpp
new file mode 100644
index 0000000..49aa67f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_09/join_q_09.2.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.
+ */
+
+/** This query is a negative test for ambiguous alias reference. */
+
+SELECT c.name AS cust_name,
+       age AS cust_age,
+       o.total AS order_total,
+       [o.oid,o.cid] AS orderList
+FROM test.Customers c, test.Orders o
+WHERE c.cid = o.cid
+ORDER BY c.name,o.total
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/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 e3d3102..2d254da 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -1349,7 +1349,18 @@
     <test-case FilePath="custord">
       <compilation-unit name="join_q_07">
         <output-dir compare="Text">join_q_06</output-dir>
-        <expected-error>Undefined alias (variable) reference for identifier c</expected-error>
+        <expected-error>Cannot find dataset c in dataverse test nor an alias with name c</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_08">
+        <output-dir compare="Text">join_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_09">
+        <output-dir compare="Text">join_q_01</output-dir>
+        <expected-error>Cannot resolve ambiguous alias (variable) reference for identifier age</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="custord">
@@ -2855,7 +2866,7 @@
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
         <expected-error>java.lang.NullPointerException</expected-error>
         <expected-error>Cannot find dataset</expected-error>
-        <expected-error>Undefined alias (variable) reference for identifier testds</expected-error>
+        <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
@@ -6666,7 +6677,7 @@
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="udf30">
         <output-dir compare="Text">udf30</output-dir>
-        <expected-error>Undefined alias (variable) reference for identifier y</expected-error>
+        <expected-error>Cannot find dataset y because there is no dataverse declared, nor an alias with name y</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/74fec83d/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 7082c25..d250e08 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -26,8 +26,10 @@ import org.apache.asterix.common.config.MetadataConstants;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.FieldAccessor;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
@@ -41,8 +43,8 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopingVisitor {
 
-    protected final FunctionSignature datasetFunction = new FunctionSignature(MetadataConstants.METADATA_DATAVERSE_NAME,
-            "dataset", 1);
+    protected final FunctionSignature datasetFunction =
+            new FunctionSignature(MetadataConstants.METADATA_DATAVERSE_NAME, "dataset", 1);
     protected final boolean overwrite;
     protected final AqlMetadataProvider metadataProvider;
 
@@ -62,25 +64,72 @@ public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopi
     }
 
     @Override
+    public Expression visit(FieldAccessor fa, ILangExpression arg) throws AsterixException {
+        Expression leadingExpr = fa.getExpr();
+        if (leadingExpr.getKind() != Kind.VARIABLE_EXPRESSION) {
+            fa.setExpr(leadingExpr.accept(this, fa));
+            return fa;
+        } else {
+            VariableExpr varExpr = (VariableExpr) leadingExpr;
+            String lastIdentifier = fa.getIdent().getValue();
+            Expression resolvedExpr = resolve(varExpr,
+                    /** Resolves within the dataverse that has the same name as the variable name. */
+                    SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue(), lastIdentifier,
+                    arg);
+            if (resolvedExpr.getKind() == Kind.CALL_EXPRESSION) {
+                CallExpr callExpr = (CallExpr) resolvedExpr;
+                if (callExpr.getFunctionSignature().equals(datasetFunction)) {
+                    // The field access is resolved to be a dataset access in the form of "dataverse.dataset".
+                    return resolvedExpr;
+                }
+            }
+            fa.setExpr(resolvedExpr);
+            return fa;
+        }
+    }
+
+    @Override
     public Expression visit(VariableExpr varExpr, ILangExpression arg) throws AsterixException {
+        return resolve(varExpr, null /** Resolves within the default dataverse. */
+                , SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue(), arg);
+    }
+
+    // Resolve a variable expression with dataverse name and dataset name.
+    private Expression resolve(VariableExpr varExpr, String dataverseName, String datasetName, ILangExpression arg)
+            throws AsterixException {
         String varName = varExpr.getVar().getValue();
-        if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new AsterixException(
-                    "Inside limit clauses, it is disallowed to reference a variable having the same name"
-                            + " as any variable bound in the same scope as the limit clause.");
-        }
+        checkError(varName);
         if (!rewriteNeeded(varExpr)) {
             return varExpr;
         }
-        boolean resolveAsDataset = resolveDatasetFirst(arg)
-                && datasetExists(SqlppVariableUtil.toUserDefinedVariableName(varName).getValue());
+        Set<VariableExpr> liveVars = SqlppVariableUtil.getLiveUserDefinedVariables(scopeChecker.getCurrentScope());
+        boolean resolveAsDataset = resolveDatasetFirst(arg) && datasetExists(dataverseName, datasetName);
         if (resolveAsDataset) {
-            return wrapWithDatasetFunction(varExpr);
+            return wrapWithDatasetFunction(dataverseName, datasetName);
+        } else if (liveVars.isEmpty()) {
+            String defaultDataverseName = metadataProvider.getDefaultDataverseName();
+            if (dataverseName == null && defaultDataverseName == null) {
+                throw new AsterixException("Cannot find dataset " + datasetName
+                        + " because there is no dataverse declared, nor an alias with name " + datasetName + "!");
+            }
+            //If no available dataset nor in-scope variable to resolve to, we throw an error.
+            throw new AsterixException("Cannot find dataset " + datasetName + " in dataverse "
+                    + (dataverseName == null ? defaultDataverseName : dataverseName) + " nor an alias with name "
+                    + datasetName + "!");
         }
-        Set<VariableExpr> liveVars = SqlppVariableUtil.getLiveUserDefinedVariables(scopeChecker.getCurrentScope());
         return wrapWithResolveFunction(varExpr, liveVars);
     }
 
+    // Checks whether we need to error the variable reference, e.g., the variable is referred
+    // in a LIMIT clause.
+    private void checkError(String varName) throws AsterixException {
+        if (scopeChecker.isInForbiddenScopes(varName)) {
+            throw new AsterixException(
+                    "Inside limit clauses, it is disallowed to reference a variable having the same name"
+                            + " as any variable bound in the same scope as the limit clause.");
+        }
+    }
+
     // For From/Join/UNNEST/NEST, we resolve the undefined identifier reference as dataset reference first.
     private boolean resolveDatasetFirst(ILangExpression arg) {
         return arg != null;
@@ -101,26 +150,25 @@ public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopi
         }
     }
 
-    private Expression wrapWithDatasetFunction(VariableExpr expr) throws AsterixException {
+    private Expression wrapWithDatasetFunction(String dataverseName, String datasetName) throws AsterixException {
+        String fullyQualifiedName = dataverseName == null ? datasetName : dataverseName + "." + datasetName;
         List<Expression> argList = new ArrayList<>();
-        //Ignore the parser-generated prefix "$" for a dataset.
-        String varName = SqlppVariableUtil.toUserDefinedVariableName(expr.getVar()).getValue();
-        argList.add(new LiteralExpr(new StringLiteral(varName)));
+        argList.add(new LiteralExpr(new StringLiteral(fullyQualifiedName)));
         return new CallExpr(datasetFunction, argList);
     }
 
-    private boolean datasetExists(String name) throws AsterixException {
+    private boolean datasetExists(String dataverseName, String datasetName) throws AsterixException {
         try {
-            if (metadataProvider.findDataset(null, name) != null) {
+            if (metadataProvider.findDataset(dataverseName, datasetName) != null) {
                 return true;
             }
-            return pathDatasetExists(name);
+            return fullyQualifiedDatasetNameExists(datasetName);
         } catch (AlgebricksException e) {
             throw new AsterixException(e);
         }
     }
 
-    private boolean pathDatasetExists(String name) throws AlgebricksException {
+    private boolean fullyQualifiedDatasetNameExists(String name) throws AlgebricksException {
         if (!name.contains(".")) {
             return false;
         }