You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2018/05/30 17:44:48 UTC

[21/24] asterixdb git commit: [ASTERIXDB-2393][COMP][RT] Add source location to error messages

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 6f99330..ed73333 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -84,6 +84,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.Log
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveEditDistanceSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveListEditDistanceSearchModifierFactory;
@@ -431,6 +432,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             addKeyVarsAndExprs(optFuncExpr, keyVarList, keyExprList, context);
             // Assign operator that sets the secondary-index search-key fields.
             inputOp = new AssignOperator(keyVarList, keyExprList);
+            inputOp.setSourceLocation(dataSourceScan.getSourceLocation());
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
             inputOp.getInputs().add(new MutableObject<>(
                     OperatorManipulationUtil.deepCopy(dataSourceScan.getInputs().get(0).getValue())));
@@ -571,6 +573,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         // Change join into a select with the same condition.
         SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
                 newNullPlaceHolderVar);
+        topSelect.setSourceLocation(indexPlanRootOp.getSourceLocation());
         topSelect.getInputs().add(indexSubTree.getRootRef());
         topSelect.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(topSelect);
@@ -595,6 +598,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         indexSubTreeVar));
             }
             UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
+            unionAllOp.setSourceLocation(topOp.getSourceLocation());
             unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
             unionAllOp.getInputs().add(panicJoinRef);
             unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -606,9 +610,10 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         // The inner (build) branch of the join is the subtree with the data scan, since the result of the similarity join could potentially be big.
         // This choice may not always be the most efficient, but it seems more robust than the alternative.
         Mutable<ILogicalExpression> eqJoinConditionRef =
-                createPrimaryKeysEqJoinCondition(originalSubTreePKs, surrogateSubTreePKs);
+                createPrimaryKeysEqJoinCondition(originalSubTreePKs, surrogateSubTreePKs, topOp.getSourceLocation());
         InnerJoinOperator topEqJoin = new InnerJoinOperator(eqJoinConditionRef, originalProbeSubTreeRootRef,
                 new MutableObject<ILogicalOperator>(topOp));
+        topEqJoin.setSourceLocation(topOp.getSourceLocation());
         topEqJoin.setExecutionMode(ExecutionMode.PARTITIONED);
         joinRef.setValue(topEqJoin);
         context.computeAndSetTypeEnvironmentForOperator(topEqJoin);
@@ -690,23 +695,30 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
     }
 
     private Mutable<ILogicalExpression> createPrimaryKeysEqJoinCondition(List<LogicalVariable> originalSubTreePKs,
-            List<LogicalVariable> surrogateSubTreePKs) {
+            List<LogicalVariable> surrogateSubTreePKs, SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> eqExprs = new ArrayList<Mutable<ILogicalExpression>>();
         int numPKVars = originalSubTreePKs.size();
         for (int i = 0; i < numPKVars; i++) {
             List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-            args.add(
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(surrogateSubTreePKs.get(i))));
-            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(originalSubTreePKs.get(i))));
-            ILogicalExpression eqFunc =
+            VariableReferenceExpression surrogateSubTreePKRef =
+                    new VariableReferenceExpression(surrogateSubTreePKs.get(i));
+            surrogateSubTreePKRef.setSourceLocation(sourceLoc);
+            args.add(new MutableObject<ILogicalExpression>(surrogateSubTreePKRef));
+            VariableReferenceExpression originalSubTreePKRef =
+                    new VariableReferenceExpression(originalSubTreePKs.get(i));
+            originalSubTreePKRef.setSourceLocation(sourceLoc);
+            args.add(new MutableObject<ILogicalExpression>(originalSubTreePKRef));
+            ScalarFunctionCallExpression eqFunc =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), args);
+            eqFunc.setSourceLocation(sourceLoc);
             eqExprs.add(new MutableObject<ILogicalExpression>(eqFunc));
         }
         if (eqExprs.size() == 1) {
             return eqExprs.get(0);
         } else {
-            ILogicalExpression andFunc = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression andFunc = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND), eqExprs);
+            andFunc.setSourceLocation(sourceLoc);
             return new MutableObject<ILogicalExpression>(andFunc);
         }
     }
@@ -715,16 +727,19 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
             IOptimizableFuncExpr optFuncExpr, Index chosenIndex, Map<LogicalVariable, LogicalVariable> panicVarMap,
             IOptimizationContext context) throws AlgebricksException {
+        ILogicalOperator probeRootOp = probeSubTree.getRoot();
+        SourceLocation sourceLoc = probeRootOp.getSourceLocation();
         LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
 
         // We split the plan into two "branches", and add selections on each side.
         AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
-        replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.getRoot()));
+        replicateOp.setSourceLocation(sourceLoc);
+        replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeRootOp));
         replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
         context.computeAndSetTypeEnvironmentForOperator(replicateOp);
 
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
-        IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.getRoot());
+        IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeRootOp);
         IAType inputSearchVarType;
         if (chosenIndex.isEnforced()) {
             inputSearchVarType = optFuncExpr.getFieldType(optFuncExpr.findLogicalVar(inputSearchVar));
@@ -770,14 +785,16 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex,
             IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef,
             Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // Create select operator for removing tuples that are not filterable.
         // First determine the proper filter function and args based on the type of the input search var.
         ILogicalExpression isFilterableExpr = null;
         switch (inputSearchVarType.getTypeTag()) {
             case STRING: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(4);
-                isFilterableArgs
-                        .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                VariableReferenceExpression inputSearchVarRef = new VariableReferenceExpression(inputSearchVar);
+                inputSearchVarRef.setSourceLocation(sourceLoc);
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(
@@ -793,8 +810,9 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
             case MULTISET:
             case ARRAY:
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
-                isFilterableArgs
-                        .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                VariableReferenceExpression inputSearchVarRef = new VariableReferenceExpression(inputSearchVar);
+                inputSearchVarRef.setSourceLocation(sourceLoc);
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
                 isFilterableExpr = new ScalarFunctionCallExpression(
@@ -802,11 +820,12 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         isFilterableArgs);
                 break;
             default:
-                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
+                throw new CompilationException(ErrorCode.NO_SUPPORTED_TYPE, sourceLoc);
         }
 
         SelectOperator isFilterableSelectOp =
                 new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr), false, null);
+        isFilterableSelectOp.setSourceLocation(sourceLoc);
         isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isFilterableSelectOp);
@@ -814,10 +833,12 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
         // Select operator for removing tuples that are filterable.
         List<Mutable<ILogicalExpression>> isNotFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>();
         isNotFilterableArgs.add(new MutableObject<ILogicalExpression>(isFilterableExpr));
-        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
+        ScalarFunctionCallExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), isNotFilterableArgs);
+        isNotFilterableExpr.setSourceLocation(sourceLoc);
         SelectOperator isNotFilterableSelectOp =
                 new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr), false, null);
+        isNotFilterableSelectOp.setSourceLocation(sourceLoc);
         isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isNotFilterableSelectOp);
@@ -854,7 +875,8 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                 typeTag = ((AUnionType) type).getActualType().getTypeTag();
             }
             if (typeTag != ATypeTag.ARRAY && typeTag != ATypeTag.STRING && typeTag != ATypeTag.MULTISET) {
-                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
+                throw new CompilationException(ErrorCode.NO_SUPPORTED_TYPE,
+                        optFuncExpr.getFuncExpr().getSourceLocation());
             }
         }
         jobGenParams.setSearchKeyType(typeTag);
@@ -1246,7 +1268,7 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         index.getGramLength(), prePost, false);
             }
             default: {
-                throw CompilationException.create(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
             }
         }
     }
@@ -1291,12 +1313,12 @@ public class InvertedIndexAccessMethod implements IAccessMethod {
                         }
                     }
                     default: {
-                        throw CompilationException.create(ErrorCode.INCOMPATIBLE_SEARCH_MODIFIER, searchModifierType,
+                        throw new CompilationException(ErrorCode.INCOMPATIBLE_SEARCH_MODIFIER, searchModifierType,
                                 index.getIndexType());
                     }
                 }
             default:
-                throw CompilationException.create(ErrorCode.UNKNOWN_SEARCH_MODIFIER, searchModifierType);
+                throw new CompilationException(ErrorCode.UNKNOWN_SEARCH_MODIFIER, searchModifierType);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 7c2edb9..9673da3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -302,7 +302,8 @@ public class OptimizableOperatorSubTree {
             // Find the dataset corresponding to the datasource in the metadata.
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw CompilationException.create(ErrorCode.NO_METADATA_FOR_DATASET, datasetName);
+                throw new CompilationException(ErrorCode.NO_METADATA_FOR_DATASET, root.getSourceLocation(),
+                        datasetName);
             }
             // Get the record type for that dataset.
             IAType itemType = metadataProvider.findType(ds.getItemTypeDataverseName(), ds.getItemTypeName());
@@ -426,7 +427,7 @@ public class OptimizableOperatorSubTree {
                 break;
             case NO_DATASOURCE:
             default:
-                throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE);
+                throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE, root.getSourceLocation());
         }
     }
 
@@ -446,7 +447,7 @@ public class OptimizableOperatorSubTree {
                 return new ArrayList<>();
             case NO_DATASOURCE:
             default:
-                throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE);
+                throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE, root.getSourceLocation());
         }
     }
 
@@ -467,7 +468,8 @@ public class OptimizableOperatorSubTree {
                     return new ArrayList<>();
                 case NO_DATASOURCE:
                 default:
-                    throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE);
+                    throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE,
+                            root.getSourceLocation());
             }
         } else {
             return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index f431603..19558aa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -254,6 +254,7 @@ public class RTreeAccessMethod implements IAccessMethod {
             // The create MBR function "extracts" one field of an MBR around the given spatial object.
             AbstractFunctionCallExpression createMBR =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
+            createMBR.setSourceLocation(optFuncExpr.getFuncExpr().getSourceLocation());
             // Spatial object is the constant from the func expr we are optimizing.
             createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr));
             // The number of dimensions
@@ -274,11 +275,13 @@ public class RTreeAccessMethod implements IAccessMethod {
         if (probeSubTree == null) {
             // We are optimizing a selection query.
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
+            assignSearchKeys.setSourceLocation(dataSourceOp.getSourceLocation());
             assignSearchKeys.getInputs().add(
                     new MutableObject<>(OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
             assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
         } else {
             // We are optimizing a join, place the assign op top of the probe subtree.
+            assignSearchKeys.setSourceLocation(probeSubTree.getRoot().getSourceLocation());
             assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
             assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
             OperatorPropertiesUtil.typeOpRec(probeSubTree.getRootRef(), context);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 8bfa53a..a98a9f4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -29,6 +29,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -85,6 +87,7 @@ import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  *   This visitor inlines all nested tuple source operators in the query
@@ -218,8 +221,9 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         for (LogicalVariable keyVar : correlatedKeyVars) {
             if (!groupKeyVars.contains(keyVar)) {
                 LogicalVariable newVar = context.newVar();
-                op.getGroupByList()
-                        .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+                VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                keyVarRef.setSourceLocation(op.getSourceLocation());
+                op.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
                 addedGroupKeyMapping.put(keyVar, newVar);
             }
         }
@@ -254,7 +258,8 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         VariableUtilities.getSubplanLocalLiveVariables(op.getInputs().get(0).getValue(), inputLiveVars);
 
         // Creates a record construction assign operator.
-        Pair<ILogicalOperator, LogicalVariable> assignOpAndRecordVar = createRecordConstructorAssignOp(inputLiveVars);
+        Pair<ILogicalOperator, LogicalVariable> assignOpAndRecordVar =
+                createRecordConstructorAssignOp(inputLiveVars, op.getSourceLocation());
         ILogicalOperator assignOp = assignOpAndRecordVar.first;
         LogicalVariable recordVar = assignOpAndRecordVar.second;
         ILogicalOperator inputOp = op.getInputs().get(0).getValue();
@@ -267,41 +272,49 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         gbyOp.getInputs().add(new MutableObject<>(assignOp));
 
         // Adds an unnest operators on top of the group-by operator.
-        Pair<ILogicalOperator, LogicalVariable> unnestOpAndUnnestVar = createUnnestForAggregatedList(aggVar);
+        Pair<ILogicalOperator, LogicalVariable> unnestOpAndUnnestVar =
+                createUnnestForAggregatedList(aggVar, op.getSourceLocation());
         ILogicalOperator unnestOp = unnestOpAndUnnestVar.first;
         LogicalVariable unnestVar = unnestOpAndUnnestVar.second;
         unnestOp.getInputs().add(new MutableObject<>(gbyOp));
 
         // Adds field accesses to recover input live variables.
-        ILogicalOperator fieldAccessAssignOp = createFieldAccessAssignOperator(unnestVar, inputLiveVars);
+        ILogicalOperator fieldAccessAssignOp =
+                createFieldAccessAssignOperator(unnestVar, inputLiveVars, op.getSourceLocation());
         fieldAccessAssignOp.getInputs().add(new MutableObject<>(unnestOp));
 
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(fieldAccessAssignOp, context);
         return fieldAccessAssignOp;
     }
 
-    private Pair<ILogicalOperator, LogicalVariable> createRecordConstructorAssignOp(
-            Set<LogicalVariable> inputLiveVars) {
+    private Pair<ILogicalOperator, LogicalVariable> createRecordConstructorAssignOp(Set<LogicalVariable> inputLiveVars,
+            SourceLocation sourceLoc) {
         // Creates a nested record.
         List<Mutable<ILogicalExpression>> recordConstructorArgs = new ArrayList<>();
         for (LogicalVariable inputLiveVar : inputLiveVars) {
             if (!correlatedKeyVars.contains(inputLiveVar)) {
                 recordConstructorArgs.add(new MutableObject<>(new ConstantExpression(
                         new AsterixConstantValue(new AString(Integer.toString(inputLiveVar.getId()))))));
-                recordConstructorArgs.add(new MutableObject<>(new VariableReferenceExpression(inputLiveVar)));
+                VariableReferenceExpression inputLiveVarRef = new VariableReferenceExpression(inputLiveVar);
+                inputLiveVarRef.setSourceLocation(sourceLoc);
+                recordConstructorArgs.add(new MutableObject<>(inputLiveVarRef));
             }
         }
         LogicalVariable recordVar = context.newVar();
-        Mutable<ILogicalExpression> recordExprRef =
-                new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), recordConstructorArgs));
+        ScalarFunctionCallExpression openRecConstr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), recordConstructorArgs);
+        openRecConstr.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> recordExprRef = new MutableObject<ILogicalExpression>(openRecConstr);
         AssignOperator assignOp = new AssignOperator(recordVar, recordExprRef);
+        assignOp.setSourceLocation(sourceLoc);
         return new Pair<>(assignOp, recordVar);
     }
 
     private Pair<ILogicalOperator, LogicalVariable> wrapLimitInGroupBy(ILogicalOperator op, LogicalVariable recordVar,
             Set<LogicalVariable> inputLiveVars) throws AlgebricksException {
+        SourceLocation sourceLoc = op.getSourceLocation();
         GroupByOperator gbyOp = new GroupByOperator();
+        gbyOp.setSourceLocation(sourceLoc);
         List<Pair<LogicalVariable, LogicalVariable>> keyVarNewVarPairs = new ArrayList<>();
         for (LogicalVariable keyVar : correlatedKeyVars) {
             // This limits the visitor can only be applied to a nested logical
@@ -309,8 +322,9 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
             // where the keyVarsToEnforce forms a candidate key which can
             // uniquely identify a tuple out of the nested-tuple-source.
             LogicalVariable newVar = context.newVar();
-            gbyOp.getGroupByList()
-                    .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(sourceLoc);
+            gbyOp.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
             keyVarNewVarPairs.add(new Pair<>(keyVar, newVar));
         }
 
@@ -322,11 +336,15 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         List<Mutable<ILogicalExpression>> aggArgList = new ArrayList<>();
         aggVarList.add(aggVar);
         // Creates an aggregation function expression.
-        aggArgList.add(new MutableObject<>(new VariableReferenceExpression(recordVar)));
-        ILogicalExpression aggExpr = new AggregateFunctionCallExpression(
+        VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+        recordVarRef.setSourceLocation(sourceLoc);
+        aggArgList.add(new MutableObject<>(recordVarRef));
+        AggregateFunctionCallExpression aggExpr = new AggregateFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false, aggArgList);
+        aggExpr.setSourceLocation(sourceLoc);
         aggExprList.add(new MutableObject<>(aggExpr));
         AggregateOperator aggOp = new AggregateOperator(aggVarList, aggExprList);
+        aggOp.setSourceLocation(sourceLoc);
 
         // Adds the original limit operator as the input operator to the added
         // aggregate operator.
@@ -335,6 +353,7 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         ILogicalOperator currentOp = op;
         if (!orderingExprs.isEmpty()) {
             OrderOperator orderOp = new OrderOperator(cloneOrderingExpression(orderingExprs));
+            orderOp.setSourceLocation(sourceLoc);
             op.getInputs().add(new MutableObject<>(orderOp));
             currentOp = orderOp;
         }
@@ -342,6 +361,7 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         // Adds a nested tuple source operator as the input operator to the
         // limit operator.
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(gbyOp));
+        nts.setSourceLocation(sourceLoc);
         currentOp.getInputs().add(new MutableObject<>(nts));
 
         // Sets the root of the added nested plan to the aggregate operator.
@@ -358,20 +378,25 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         return new Pair<>(gbyOp, aggVar);
     }
 
-    private Pair<ILogicalOperator, LogicalVariable> createUnnestForAggregatedList(LogicalVariable aggVar) {
+    private Pair<ILogicalOperator, LogicalVariable> createUnnestForAggregatedList(LogicalVariable aggVar,
+            SourceLocation sourceLoc) {
         LogicalVariable unnestVar = context.newVar();
         // Creates an unnest function expression.
-        Mutable<ILogicalExpression> unnestArg = new MutableObject<>(new VariableReferenceExpression(aggVar));
+        VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+        aggVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> unnestArg = new MutableObject<>(aggVarRef);
         List<Mutable<ILogicalExpression>> unnestArgList = new ArrayList<>();
         unnestArgList.add(unnestArg);
-        Mutable<ILogicalExpression> unnestExpr = new MutableObject<>(new UnnestingFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), unnestArgList));
-        ILogicalOperator unnestOp = new UnnestOperator(unnestVar, unnestExpr);
+        UnnestingFunctionCallExpression unnestExpr = new UnnestingFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), unnestArgList);
+        unnestExpr.setSourceLocation(sourceLoc);
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
+        unnestOp.setSourceLocation(sourceLoc);
         return new Pair<>(unnestOp, unnestVar);
     }
 
     private ILogicalOperator createFieldAccessAssignOperator(LogicalVariable recordVar,
-            Set<LogicalVariable> inputLiveVars) {
+            Set<LogicalVariable> inputLiveVars, SourceLocation sourceLoc) {
         List<LogicalVariable> fieldAccessVars = new ArrayList<>();
         List<Mutable<ILogicalExpression>> fieldAccessExprs = new ArrayList<>();
         // Adds field access by name.
@@ -382,16 +407,22 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
                 fieldAccessVars.add(newVar);
                 // fieldAcess expr
                 List<Mutable<ILogicalExpression>> argRefs = new ArrayList<>();
-                argRefs.add(new MutableObject<>(new VariableReferenceExpression(recordVar)));
+                VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+                recordVarRef.setSourceLocation(sourceLoc);
+                argRefs.add(new MutableObject<>(recordVarRef));
                 argRefs.add(new MutableObject<>(new ConstantExpression(
                         new AsterixConstantValue(new AString(Integer.toString(inputLiveVar.getId()))))));
-                fieldAccessExprs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argRefs)));
+                ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argRefs);
+                faExpr.setSourceLocation(sourceLoc);
+                fieldAccessExprs.add(new MutableObject<>(faExpr));
                 // Updates variable mapping for ancestor operators.
                 updateInputToOutputVarMapping(inputLiveVar, newVar, false);
             }
         }
-        return new AssignOperator(fieldAccessVars, fieldAccessExprs);
+        AssignOperator assignOp = new AssignOperator(fieldAccessVars, fieldAccessExprs);
+        assignOp.setSourceLocation(sourceLoc);
+        return assignOp;
     }
 
     @Override
@@ -445,13 +476,15 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>();
         // Adds keyVars to the prefix of sorting columns.
         for (LogicalVariable keyVar : correlatedKeyVars) {
-            orderExprList.add(
-                    new Pair<>(OrderOperator.ASC_ORDER, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(op.getSourceLocation());
+            orderExprList.add(new Pair<>(OrderOperator.ASC_ORDER, new MutableObject<>(keyVarRef)));
         }
         orderExprList.addAll(op.getOrderExpressions());
 
         // Creates an order operator with the new expression list.
         OrderOperator orderOp = new OrderOperator(orderExprList);
+        orderOp.setSourceLocation(op.getSourceLocation());
         orderOp.getInputs().addAll(op.getInputs());
         context.computeAndSetTypeEnvironmentForOperator(orderOp);
         return orderOp;
@@ -542,7 +575,8 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         for (int i = 0; i < op.getNumInput(); i++) {
             List<LogicalVariable> inputVars = op.getInputVariables(i);
             if (inputVars.size() != outputVars.size()) {
-                throw new AlgebricksException("The cardinality of input and output are not equal for Intersection");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                        "The cardinality of input and output are not equal for Intersection");
             }
             for (int j = 0; j < inputVars.size(); j++) {
                 updateInputToOutputVarMapping(inputVars.get(j), outputVars.get(j), false);
@@ -576,7 +610,7 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
     @Override
     public ILogicalOperator visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void arg)
             throws AlgebricksException {
-        throw new AlgebricksException(
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
                 "The subquery de-correlation rule should always be applied before index-access-method related rules.");
     }
 
@@ -626,7 +660,9 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         if (correlatedKeyVars.isEmpty()) {
             return op;
         }
+        SourceLocation sourceLoc = op.getSourceLocation();
         GroupByOperator gbyOp = new GroupByOperator();
+        gbyOp.setSourceLocation(sourceLoc);
         // Creates a copy of correlatedKeyVars, to fix the ConcurrentModificationException in ASTERIXDB-1581.
         List<LogicalVariable> copyOfCorrelatedKeyVars = new ArrayList<>(correlatedKeyVars);
         for (LogicalVariable keyVar : copyOfCorrelatedKeyVars) {
@@ -635,8 +671,9 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
             // where the keyVarsToEnforce forms a candidate key which can
             // uniquely identify a tuple out of the nested-tuple-source.
             LogicalVariable newVar = context.newVar();
-            gbyOp.getGroupByList()
-                    .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(sourceLoc);
+            gbyOp.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
             updateInputToOutputVarMapping(keyVar, newVar, false);
         }
 
@@ -644,6 +681,7 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         gbyOp.getInputs().add(new MutableObject<>(inputOp));
 
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(gbyOp));
+        nts.setSourceLocation(sourceLoc);
         op.getInputs().clear();
         op.getInputs().add(new MutableObject<>(nts));
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 4a28344..610fea2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -192,6 +192,7 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
         // After rewriting, the original inner join should become an left outer join.
         if (rewritten) {
             returnOp = new LeftOuterJoinOperator(op.getCondition());
+            returnOp.setSourceLocation(op.getSourceLocation());
             returnOp.getInputs().addAll(op.getInputs());
             injectNullCheckVars(returnOp);
         }
@@ -231,13 +232,16 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>();
         // Adds keyVars to the prefix of sorting columns.
         for (LogicalVariable liveVar : liveVarsFromSubplanInput) {
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(op.getSourceLocation());
             orderExprList.add(new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER,
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(liveVar))));
+                    new MutableObject<ILogicalExpression>(liveVarRef)));
         }
         orderExprList.addAll(op.getOrderExpressions());
 
         // Creates an order operator with the new expression list.
         OrderOperator orderOp = new OrderOperator(orderExprList);
+        orderOp.setSourceLocation(op.getSourceLocation());
         orderOp.getInputs().addAll(op.getInputs());
         context.computeAndSetTypeEnvironmentForOperator(orderOp);
         return orderOp;
@@ -399,8 +403,9 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
      */
     private void injectNullCheckVars(AbstractBinaryJoinOperator joinOp) {
         LogicalVariable assignVar = context.newVar();
-        ILogicalOperator assignOp =
+        AssignOperator assignOp =
                 new AssignOperator(assignVar, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        assignOp.setSourceLocation(joinOp.getSourceLocation());
         assignOp.getInputs().add(joinOp.getInputs().get(1));
         joinOp.getInputs().set(1, new MutableObject<ILogicalOperator>(assignOp));
         nullCheckVars.add(assignVar);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index 3edccec..62f89f1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -56,6 +56,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.Var
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /*
 This rule  is to remove SubplanOperators containing DataScan, InnerJoin, LeftOuterJoin.
@@ -356,6 +357,7 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
                 LogicalOperatorTag.INNERJOIN, LogicalOperatorTag.LEFTOUTERJOIN))) {
             return new Pair<>(false, new LinkedHashMap<>());
         }
+        SourceLocation sourceLoc = subplanOp.getSourceLocation();
         Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
         ILogicalOperator inputOpBackup = inputOpRef.getValue();
         // Creates parameters for the left outer join operator.
@@ -382,7 +384,8 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
 
         // Creates a variable to indicate whether a left input tuple is killed in the plan rooted at rightInputOp.
         LogicalVariable assignVar = context.newVar();
-        ILogicalOperator assignOp = new AssignOperator(assignVar, new MutableObject<>(ConstantExpression.TRUE));
+        AssignOperator assignOp = new AssignOperator(assignVar, new MutableObject<>(ConstantExpression.TRUE));
+        assignOp.setSourceLocation(rightInputOp.getSourceLocation());
         assignOp.getInputs().add(rightInputOpRef);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         rightInputOpRef = new MutableObject<>(assignOp);
@@ -391,20 +394,31 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
         List<Mutable<ILogicalExpression>> joinPredicates = new ArrayList<>();
         for (LogicalVariable liveVar : primaryKeyVars) {
             List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
-            arguments.add(new MutableObject<>(new VariableReferenceExpression(liveVar)));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            arguments.add(new MutableObject<>(liveVarRef));
             LogicalVariable rightVar = varMap.get(liveVar);
-            arguments.add(new MutableObject<>(new VariableReferenceExpression(rightVar)));
-            ILogicalExpression expr = new ScalarFunctionCallExpression(
+            VariableReferenceExpression rightVarRef = new VariableReferenceExpression(rightVar);
+            rightVarRef.setSourceLocation(sourceLoc);
+            arguments.add(new MutableObject<>(rightVarRef));
+            ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), arguments);
+            expr.setSourceLocation(sourceLoc);
             joinPredicates.add(new MutableObject<>(expr));
         }
 
-        ILogicalExpression joinExpr = joinPredicates.size() > 1
-                ? new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND),
-                        joinPredicates)
-                : joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
+        ILogicalExpression joinExpr;
+        if (joinPredicates.size() > 1) {
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND), joinPredicates);
+            andExpr.setSourceLocation(sourceLoc);
+            joinExpr = andExpr;
+        } else {
+            joinExpr = joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
+        }
         LeftOuterJoinOperator leftOuterJoinOp =
                 new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef, rightInputOpRef);
+        leftOuterJoinOp.setSourceLocation(sourceLoc);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rightInputOp, context);
         context.computeAndSetTypeEnvironmentForOperator(leftOuterJoinOp);
 
@@ -413,11 +427,14 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByDecorList = new ArrayList<>();
         List<ILogicalPlan> nestedPlans = new ArrayList<>();
         GroupByOperator groupbyOp = new GroupByOperator(groupByList, groupByDecorList, nestedPlans);
+        groupbyOp.setSourceLocation(sourceLoc);
 
         LinkedHashMap<LogicalVariable, LogicalVariable> replacedVarMap = new LinkedHashMap<>();
         for (LogicalVariable liveVar : primaryKeyVars) {
             LogicalVariable newVar = context.newVar();
-            groupByList.add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(inputOpBackup.getSourceLocation());
+            groupByList.add(new Pair<>(newVar, new MutableObject<>(liveVarRef)));
             // Adds variables for replacements in ancestors.
             replacedVarMap.put(liveVar, newVar);
         }
@@ -425,7 +442,9 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
             if (primaryKeyVars.contains(liveVar)) {
                 continue;
             }
-            groupByDecorList.add(new Pair<>(null, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            groupByDecorList.add(new Pair<>(null, new MutableObject<>(liveVarRef)));
         }
 
         // Sets up the nested plan for the groupby operator.
@@ -436,23 +455,33 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprs = varMapAndOrderExprs.second;
         if (!orderExprs.isEmpty()) {
             OrderOperator orderOp = new OrderOperator(orderExprs);
+            orderOp.setSourceLocation(sourceLoc);
             currentOpRef = new MutableObject<>(orderOp);
             lowestAggregateRefInSubplan.getValue().getInputs().add(currentOpRef);
         }
 
         // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
         // i.e., subplan input tuples that are filtered out within a subplan.
-        Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(new VariableReferenceExpression(assignVar));
+        VariableReferenceExpression assignVarRef = new VariableReferenceExpression(assignVar);
+        assignVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(assignVarRef);
         List<Mutable<ILogicalExpression>> args = new ArrayList<>();
         args.add(filterVarExpr);
         List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-        argsForNotFunction.add(new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args)));
-        SelectOperator selectOp = new SelectOperator(new MutableObject<>(new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction)), false, null);
+        ScalarFunctionCallExpression isMissingExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+        isMissingExpr.setSourceLocation(sourceLoc);
+        argsForNotFunction.add(new MutableObject<>(isMissingExpr));
+        ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
+        notExpr.setSourceLocation(sourceLoc);
+        SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr), false, null);
+        selectOp.setSourceLocation(sourceLoc);
         currentOpRef.getValue().getInputs().add(new MutableObject<>(selectOp));
 
-        selectOp.getInputs().add(new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp))));
+        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+        ntsOp.setSourceLocation(sourceLoc);
+        selectOp.getInputs().add(new MutableObject<>(ntsOp));
         List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<>();
         nestedRoots.add(aggOpRef);
         nestedPlans.add(new ALogicalPlanImpl(nestedRoots));
@@ -476,6 +505,7 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
     private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applySpecialFlattening(
             Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
+        SourceLocation sourceLoc = subplanOp.getSourceLocation();
         Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
         LinkedHashMap<LogicalVariable, LogicalVariable> replacedVarMap = new LinkedHashMap<>();
 
@@ -508,10 +538,13 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByList = new ArrayList<>();
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByDecorList = new ArrayList<>();
         GroupByOperator groupbyOp = new GroupByOperator(groupByList, groupByDecorList, subplanOp.getNestedPlans());
+        groupbyOp.setSourceLocation(sourceLoc);
 
         for (LogicalVariable coverVar : primaryKeyVars) {
             LogicalVariable newVar = context.newVar();
-            groupByList.add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(coverVar))));
+            VariableReferenceExpression coverVarRef = new VariableReferenceExpression(coverVar);
+            coverVarRef.setSourceLocation(sourceLoc);
+            groupByList.add(new Pair<>(newVar, new MutableObject<>(coverVarRef)));
             // Adds variables for replacements in ancestors.
             replacedVarMap.put(coverVar, newVar);
         }
@@ -519,7 +552,9 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
             if (primaryKeyVars.contains(liveVar)) {
                 continue;
             }
-            groupByDecorList.add(new Pair<>(null, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            groupByDecorList.add(new Pair<>(null, new MutableObject<>(liveVarRef)));
         }
         groupbyOp.getInputs().add(new MutableObject<>(topJoinRef.getValue()));
 
@@ -528,28 +563,42 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
             // i.e., subplan input tuples that are filtered out within a subplan.
             List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
             for (LogicalVariable notNullVar : notNullVars) {
-                Mutable<ILogicalExpression> filterVarExpr =
-                        new MutableObject<>(new VariableReferenceExpression(notNullVar));
+                VariableReferenceExpression notNullVarRef = new VariableReferenceExpression(notNullVar);
+                notNullVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(notNullVarRef);
                 List<Mutable<ILogicalExpression>> args = new ArrayList<>();
                 args.add(filterVarExpr);
                 List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-                argsForNotFunction.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args)));
-                nullCheckExprRefs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction)));
+                ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+                isMissingExpr.setSourceLocation(sourceLoc);
+                argsForNotFunction.add(new MutableObject<>(isMissingExpr));
+                ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
+                notExpr.setSourceLocation(sourceLoc);
+                nullCheckExprRefs.add(new MutableObject<>(notExpr));
+            }
+            Mutable<ILogicalExpression> selectExprRef;
+            if (nullCheckExprRefs.size() > 1) {
+                ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), nullCheckExprRefs);
+                andExpr.setSourceLocation(sourceLoc);
+                selectExprRef = new MutableObject<>(andExpr);
+            } else {
+                selectExprRef = nullCheckExprRefs.get(0);
             }
-            Mutable<ILogicalExpression> selectExprRef = nullCheckExprRefs.size() > 1
-                    ? new MutableObject<>(new ScalarFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), nullCheckExprRefs))
-                    : nullCheckExprRefs.get(0);
             SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+            selectOp.setSourceLocation(sourceLoc);
             topJoinRef.setValue(selectOp);
-            selectOp.getInputs()
-                    .add(new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp))));
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            selectOp.getInputs().add(new MutableObject<>(ntsOp));
         } else {
             // The original join operator in the Subplan is a left-outer join.
             // Therefore, no null-check variable is injected and no SelectOperator needs to be added.
-            topJoinRef.setValue(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp)));
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            topJoinRef.setValue(ntsOp);
         }
         opRef.setValue(groupbyOp);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(groupbyOp, context);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
index 0d53a19..5b85cb6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
@@ -212,7 +212,10 @@ public class TranslateIntervalExpressionRule implements IAlgebraicRewriteRule {
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval) {
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
         intervalArg.add(new MutableObject<ILogicalExpression>(interval));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        ScalarFunctionCallExpression fnExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        fnExpr.setSourceLocation(interval.getSourceLocation());
+        return fnExpr;
     }
 
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval1,
@@ -220,7 +223,10 @@ public class TranslateIntervalExpressionRule implements IAlgebraicRewriteRule {
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
         intervalArg.add(new MutableObject<ILogicalExpression>(interval1));
         intervalArg.add(new MutableObject<ILogicalExpression>(interval2));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        ScalarFunctionCallExpression fnExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        fnExpr.setSourceLocation(interval1.getSourceLocation());
+        return fnExpr;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index efea208..4b2ff76 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -25,6 +25,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AString;
@@ -165,7 +167,7 @@ public class StaticTypeCastUtil {
                 }
             }
             if (!compatible(reqType, inputType)) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, funcExpr.getSourceLocation(),
                         "type mismatch, required: " + reqType.toString() + " actual: " + inputType.toString());
             }
             return changed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
index 82d963d..1593cf1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
@@ -116,11 +116,13 @@ public class EquivalenceClassUtils {
                 fieldIndexInRecord = metaFieldNameToIndexMap.get(pkFieldName);
             }
             LogicalVariable var = indexSearchVars.get(pkIndex);
-            ILogicalExpression expr = new ScalarFunctionCallExpression(
+            VariableReferenceExpression referredRecordVarRef = new VariableReferenceExpression(referredRecordVar);
+            referredRecordVarRef.setSourceLocation(operator.getSourceLocation());
+            ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(referredRecordVar)),
-                    new MutableObject<ILogicalExpression>(
+                    new MutableObject<ILogicalExpression>(referredRecordVarRef), new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(fieldIndexInRecord)))));
+            expr.setSourceLocation(operator.getSourceLocation());
             EquivalenceClass equivClass =
                     new EquivalenceClass(Collections.singletonList(var), var, Collections.singletonList(expr));
             Map<LogicalVariable, EquivalenceClass> equivalenceMap = context.getEquivalenceClassMap(operator);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
index 81c5f4c..e777c4e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -170,7 +170,7 @@ class AqlExpressionToPlanTranslator extends LangExpressionToPlanTranslator imple
             throws CompilationException {
         List<ILangExpression> inputExprs = new ArrayList<>();
         inputExprs.addAll(unionExpr.getExprs());
-        Pair<ILogicalOperator, LogicalVariable> result = translateUnionAllFromInputExprs(inputExprs, tupSource);
+        Pair<ILogicalOperator, LogicalVariable> result = translateUnionAllFromInputExprs(inputExprs, tupSource, null);
         return aggListifyForSubquery(result.second, new MutableObject<>(result.first), false);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 3a4682e..5950329 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -28,6 +28,7 @@ import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * An AQL statement instance is translated into an instance of type CompileX
@@ -36,11 +37,24 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 public class CompiledStatements {
 
     public interface ICompiledStatement {
-
         Statement.Kind getKind();
+
+        SourceLocation getSourceLocation();
+    }
+
+    public static abstract class AbstractCompiledStatement implements ICompiledStatement {
+        private SourceLocation sourceLoc;
+
+        public void setSourceLocation(SourceLocation sourceLoc) {
+            this.sourceLoc = sourceLoc;
+        }
+
+        public SourceLocation getSourceLocation() {
+            return sourceLoc;
+        }
     }
 
-    public static class CompiledDatasetDropStatement implements ICompiledStatement {
+    public static class CompiledDatasetDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
 
@@ -64,7 +78,7 @@ public class CompiledStatements {
     }
 
     // added by yasser
-    public static class CompiledCreateDataverseStatement implements ICompiledStatement {
+    public static class CompiledCreateDataverseStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String format;
 
@@ -87,7 +101,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+    public static class CompiledNodeGroupDropStatement extends AbstractCompiledStatement {
         private final String nodeGroupName;
 
         public CompiledNodeGroupDropStatement(String nodeGroupName) {
@@ -104,7 +118,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledIndexDropStatement implements ICompiledStatement {
+    public static class CompiledIndexDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
         private final String indexName;
@@ -133,7 +147,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledDataverseDropStatement implements ICompiledStatement {
+    public static class CompiledDataverseDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final boolean ifExists;
 
@@ -156,7 +170,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledTypeDropStatement implements ICompiledStatement {
+    public static class CompiledTypeDropStatement extends AbstractCompiledStatement {
         private final String typeName;
 
         public CompiledTypeDropStatement(String nodeGroupName) {
@@ -173,14 +187,15 @@ public class CompiledStatements {
         }
     }
 
-    public static interface ICompiledDmlStatement extends ICompiledStatement {
+    public interface ICompiledDmlStatement extends ICompiledStatement {
 
-        public String getDataverseName();
+        String getDataverseName();
 
-        public String getDatasetName();
+        String getDatasetName();
     }
 
-    public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
+    public static class CompiledCreateIndexStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
         private final Dataset dataset;
         private final Index index;
 
@@ -213,7 +228,8 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledLoadFromFileStatement implements ICompiledDmlStatement {
+    public static class CompiledLoadFromFileStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final boolean alreadySorted;
@@ -257,7 +273,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledInsertStatement implements ICompiledDmlStatement {
+    public static class CompiledInsertStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final Query query;
@@ -320,7 +336,8 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledSubscribeFeedStatement implements ICompiledDmlStatement {
+    public static class CompiledSubscribeFeedStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
 
         private FeedConnectionRequest request;
         private final int varCounter;
@@ -354,7 +371,7 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledDeleteStatement implements ICompiledDmlStatement {
+    public static class CompiledDeleteStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final Expression condition;
@@ -399,7 +416,7 @@ public class CompiledStatements {
 
     }
 
-    public static class CompiledCompactStatement implements ICompiledStatement {
+    public static class CompiledCompactStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;