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:50 UTC

[23/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/FullTextContainsParameterCheckRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
index a93b732..9a44022 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
@@ -44,6 +44,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogi
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Checks whether the given parameters of the ftcontains() function are correct during the compilation.
@@ -144,8 +145,8 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
 
                 // The number of parameters should be three: exp1, exp2, and the option
                 if (oldExprs.size() != numberOfCorrectArguments) {
-                    throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, fi,
-                            oldExprs.size());
+                    throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
+                            funcExpr.getSourceLocation(), fi, oldExprs.size());
                 }
 
                 // The last expression before the option needs to be copied first.
@@ -184,8 +185,8 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
             ILogicalExpression firstExpr = exprs.get(0).getValue();
             if (firstExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
                     && ConstantExpressionUtil.getConstantIaObjectType(firstExpr) != ATypeTag.STRING) {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName,
-                        ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, firstExpr.getSourceLocation(),
+                        functionName, ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
             }
 
             // Check the second parameter - Expression2. If it's a constant, then we can check the type here.
@@ -198,7 +199,8 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                     case ARRAY:
                         break;
                     default:
-                        throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, exprTypeTag);
+                        throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, secondExpr.getSourceLocation(),
+                                functionName, exprTypeTag);
                 }
             }
         }
@@ -216,12 +218,14 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
             FunctionIdentifier openRecConsFi = openRecConsExpr.getFunctionIdentifier();
             if (openRecConsFi != BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR
                     && openRecConsFi != BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR) {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, openRecConsFi);
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, openRecConsExpr.getSourceLocation(),
+                        functionName, openRecConsFi);
             }
 
             // We multiply 2 because the layout of the arguments are: [expr, val, expr1, val1, ...]
             if (openRecConsExpr.getArguments().size() > FullTextContainsDescriptor.getParamTypeMap().size() * 2) {
-                throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION, functionName);
+                throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION,
+                        openRecConsExpr.getSourceLocation(), functionName);
             }
 
             for (int i = 0; i < openRecConsExpr.getArguments().size(); i = i + 2) {
@@ -231,13 +235,14 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                 String option = ConstantExpressionUtil.getStringConstant(optionExpr);
 
                 if (optionExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT || option == null) {
-                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName,
-                            optionExpr.getExpressionTag());
+                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExpr.getSourceLocation(),
+                            functionName, optionExpr.getExpressionTag());
                 }
 
                 option = option.toLowerCase();
                 if (!FullTextContainsDescriptor.getParamTypeMap().containsKey(option)) {
-                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExprVal.getSourceLocation(),
+                            functionName, option);
                 }
 
                 String optionTypeStringVal = null;
@@ -248,22 +253,25 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
                         case STRING:
                             optionTypeStringVal = ConstantExpressionUtil.getStringConstant(optionExprVal);
                             if (optionTypeStringVal == null) {
-                                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                        optionExprVal.getSourceLocation(), functionName, option);
                             }
                             optionTypeStringVal = optionTypeStringVal.toLowerCase();
                             break;
                         default:
                             // Currently, we only have a string parameter. So, the flow doesn't reach here.
-                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                    optionExprVal.getSourceLocation(), functionName, option);
                     }
 
                     // Check the validity of option value
                     switch (option) {
                         case FullTextContainsDescriptor.SEARCH_MODE_OPTION:
-                            checkSearchModeOption(optionTypeStringVal, functionName);
+                            checkSearchModeOption(optionTypeStringVal, functionName, optionExprVal.getSourceLocation());
                             break;
                         default:
-                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                    optionExprVal.getSourceLocation(), functionName, option);
                     }
                 }
 
@@ -273,12 +281,13 @@ public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule
             }
         }
 
-        private void checkSearchModeOption(String optionVal, String functionName) throws AlgebricksException {
+        private void checkSearchModeOption(String optionVal, String functionName, SourceLocation sourceLoc)
+                throws AlgebricksException {
             if (optionVal.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)
                     || optionVal.equals(FullTextContainsDescriptor.DISJUNCTIVE_SEARCH_MODE_OPTION)) {
                 return;
             } else {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, optionVal);
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, functionName, optionVal);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
index 511ec55..895746a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
@@ -45,6 +45,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBina
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FuzzyEqRule implements IAlgebraicRewriteRule {
 
@@ -88,6 +89,7 @@ public class FuzzyEqRule implements IAlgebraicRewriteRule {
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) exp;
         FunctionIdentifier fi = funcExp.getFunctionIdentifier();
         if (fi.equals(BuiltinFunctions.FUZZY_EQ)) {
+            SourceLocation sourceLoc = funcExp.getSourceLocation();
             List<Mutable<ILogicalExpression>> inputExps = funcExp.getArguments();
 
             String simFuncName = FuzzyUtils.getSimFunction(metadataProvider);
@@ -100,13 +102,15 @@ public class FuzzyEqRule implements IAlgebraicRewriteRule {
             FunctionIdentifier simFunctionIdentifier = FuzzyUtils.getFunctionIdentifier(simFuncName);
             ScalarFunctionCallExpression similarityExp = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(simFunctionIdentifier), similarityArgs);
+            similarityExp.setSourceLocation(sourceLoc);
             // Add annotations from the original fuzzy-eq function.
             similarityExp.getAnnotations().putAll(funcExp.getAnnotations());
             ArrayList<Mutable<ILogicalExpression>> cmpArgs = new ArrayList<Mutable<ILogicalExpression>>();
             cmpArgs.add(new MutableObject<ILogicalExpression>(similarityExp));
             IAObject simThreshold = FuzzyUtils.getSimThreshold(metadataProvider, simFuncName);
-            cmpArgs.add(new MutableObject<ILogicalExpression>(
-                    new ConstantExpression(new AsterixConstantValue(simThreshold))));
+            ConstantExpression simThresholdExpr = new ConstantExpression(new AsterixConstantValue(simThreshold));
+            simThresholdExpr.setSourceLocation(sourceLoc);
+            cmpArgs.add(new MutableObject<ILogicalExpression>(simThresholdExpr));
             ScalarFunctionCallExpression cmpExpr = FuzzyUtils.getComparisonExpr(simFuncName, cmpArgs);
             expRef.setValue(cmpExpr);
             return true;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
index 003cff0..fae10e1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -329,6 +329,7 @@ public class FuzzyJoinRule implements IAlgebraicRewriteRule {
             switch (joinOp.getJoinKind()) {
                 case INNER: {
                     extraSelect = new SelectOperator(expRef, false, null);
+                    extraSelect.setSourceLocation(expRef.getValue().getSourceLocation());
                     extraSelect.getInputs().add(new MutableObject<ILogicalOperator>(outputOp));
                     outputOp = extraSelect;
                     break;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
index 27b2712..92b2d6a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
@@ -103,6 +103,7 @@ public class InjectTypeCastForSwitchCaseRule implements IAlgebraicRewriteRule {
                 ScalarFunctionCallExpression castFunc =
                         new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
                                 new ArrayList<>(Collections.singletonList(new MutableObject<>(argExpr))));
+                castFunc.setSourceLocation(argExpr.getSourceLocation());
                 TypeCastUtils.setRequiredAndInputTypes(castFunc, producedType, type);
                 argRef.setValue(castFunc);
                 rewritten = true;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
index 748f326..b7c1854 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceE
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule injects type casts for inputs of a UnionAllOperator if those
@@ -69,6 +70,7 @@ public class InjectTypeCastForUnionRule implements IAlgebraicRewriteRule {
         IVariableTypeEnvironment env = context.getOutputTypeEnvironment(op);
         Mutable<ILogicalOperator> branchOpRef = op.getInputs().get(childIndex);
         IVariableTypeEnvironment childEnv = context.getOutputTypeEnvironment(branchOpRef.getValue());
+        SourceLocation sourceLoc = branchOpRef.getValue().getSourceLocation();
 
         // The two lists are used for the assign operator that calls cast functions.
         List<LogicalVariable> varsToCast = new ArrayList<>();
@@ -89,9 +91,12 @@ public class InjectTypeCastForUnionRule implements IAlgebraicRewriteRule {
             // Resets triple variables to new variables that bind to the results of type casting.
             triple.first = childIndex == 0 ? castedVar : triple.first;
             triple.second = childIndex > 0 ? castedVar : triple.second;
-            ScalarFunctionCallExpression castFunc = new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), new ArrayList<>(Collections
-                            .singletonList(new MutableObject<>(new VariableReferenceExpression(varToCast)))));
+            VariableReferenceExpression varToCastRef = new VariableReferenceExpression(varToCast);
+            varToCastRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression castFunc =
+                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
+                            new ArrayList<>(Collections.singletonList(new MutableObject<>(varToCastRef))));
+            castFunc.setSourceLocation(sourceLoc);
             TypeCastUtils.setRequiredAndInputTypes(castFunc, producedType, inputType);
 
             // Adds the variable and function expression into lists, for the assign operator.
@@ -103,6 +108,7 @@ public class InjectTypeCastForUnionRule implements IAlgebraicRewriteRule {
         }
         // Injects an assign operator to perform type casts.
         AssignOperator assignOp = new AssignOperator(varsToCast, castFunctionsForLeft);
+        assignOp.setSourceLocation(sourceLoc);
         assignOp.getInputs().add(new MutableObject<>(branchOpRef.getValue()));
         branchOpRef.setValue(assignOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
index 14c3d87..f1b20d8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
@@ -52,6 +52,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDelete
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
 
@@ -122,6 +123,7 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
         }
         AssignOperator assignOp = (AssignOperator) parentOp;
         LogicalVariable inputRecord;
+        SourceLocation inputRecordSourceLoc;
 
         boolean hasFilter = false;
         AbstractLogicalOperator grandparentOp = (AbstractLogicalOperator) parentOp.getInputs().get(0).getValue();
@@ -130,9 +132,11 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
         if (grandparentOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
             ProjectOperator projectOp = (ProjectOperator) grandparentOp;
             inputRecord = projectOp.getVariables().get(0);
+            inputRecordSourceLoc = projectOp.getSourceLocation();
         } else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator dssOp = (DataSourceScanOperator) grandparentOp;
             inputRecord = dssOp.getVariables().get(0);
+            inputRecordSourceLoc = dssOp.getSourceLocation();
         } else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AbstractLogicalOperator greatgrandparentOp =
                     (AbstractLogicalOperator) grandparentOp.getInputs().get(0).getValue();
@@ -142,6 +146,7 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
             //filter case
             ProjectOperator projectOp = (ProjectOperator) greatgrandparentOp;
             inputRecord = projectOp.getVariables().get(0);
+            inputRecordSourceLoc = projectOp.getSourceLocation();
             newAssignParentOp = greatgrandparentOp;
             newAssignChildOp = grandparentOp;
             hasFilter = true;
@@ -149,15 +154,18 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
             return false;
         }
 
+        SourceLocation insertOpSourceLoc = insertOp.getSourceLocation();
         List<String> pkFieldName =
                 ((InternalDatasetDetails) dds.getDataset().getDatasetDetails()).getPrimaryKey().get(0);
-        ILogicalExpression rec0 = new VariableReferenceExpression(inputRecord);
-        ILogicalExpression rec1 = createPrimaryKeyRecordExpression(pkFieldName);
-        ILogicalExpression mergedRec = createRecordMergeFunction(rec0, rec1);
+        VariableReferenceExpression rec0 = new VariableReferenceExpression(inputRecord);
+        rec0.setSourceLocation(inputRecordSourceLoc);
+        ILogicalExpression rec1 = createPrimaryKeyRecordExpression(pkFieldName, insertOpSourceLoc);
+        ILogicalExpression mergedRec = createRecordMergeFunction(rec0, rec1, insertOpSourceLoc);
         ILogicalExpression nonNullMergedRec = createNotNullFunction(mergedRec);
 
         LogicalVariable v = context.newVar();
         AssignOperator newAssign = new AssignOperator(v, new MutableObject<ILogicalExpression>(nonNullMergedRec));
+        newAssign.setSourceLocation(insertOpSourceLoc);
         newAssign.getInputs().add(new MutableObject<ILogicalOperator>(newAssignParentOp));
         newAssignChildOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
         if (hasFilter) {
@@ -184,21 +192,25 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
         args.add(new MutableObject<ILogicalExpression>(mergedRec));
         AbstractFunctionCallExpression notNullFn =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CHECK_UNKNOWN), args);
+        notNullFn.setSourceLocation(mergedRec.getSourceLocation());
         return notNullFn;
     }
 
-    private AbstractFunctionCallExpression createPrimaryKeyRecordExpression(List<String> pkFieldName) {
+    private AbstractFunctionCallExpression createPrimaryKeyRecordExpression(List<String> pkFieldName,
+            SourceLocation sourceLoc) {
         //Create lowest level of nested uuid
         AbstractFunctionCallExpression uuidFn =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_UUID));
+        uuidFn.setSourceLocation(sourceLoc);
         List<Mutable<ILogicalExpression>> openRecordConsArgs = new ArrayList<>();
-        Mutable<ILogicalExpression> pkFieldNameExpression = new MutableObject<ILogicalExpression>(
-                new ConstantExpression(new AsterixConstantValue(new AString(pkFieldName.get(pkFieldName.size() - 1)))));
-        openRecordConsArgs.add(pkFieldNameExpression);
-        Mutable<ILogicalExpression> pkFieldValueExpression = new MutableObject<ILogicalExpression>(uuidFn);
-        openRecordConsArgs.add(pkFieldValueExpression);
+        ConstantExpression pkFieldNameExpression =
+                new ConstantExpression(new AsterixConstantValue(new AString(pkFieldName.get(pkFieldName.size() - 1))));
+        pkFieldNameExpression.setSourceLocation(sourceLoc);
+        openRecordConsArgs.add(new MutableObject<>(pkFieldNameExpression));
+        openRecordConsArgs.add(new MutableObject<>(uuidFn));
         AbstractFunctionCallExpression openRecFn = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), openRecordConsArgs);
+        openRecFn.setSourceLocation(sourceLoc);
 
         //Create higher levels
         for (int i = pkFieldName.size() - 2; i > -1; i--) {
@@ -209,17 +221,20 @@ public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
             openRecordConsArgs.add(new MutableObject<ILogicalExpression>(openRecFn));
             openRecFn = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), openRecordConsArgs);
+            openRecFn.setSourceLocation(sourceLoc);
         }
 
         return openRecFn;
     }
 
-    private AbstractFunctionCallExpression createRecordMergeFunction(ILogicalExpression rec0, ILogicalExpression rec1) {
+    private AbstractFunctionCallExpression createRecordMergeFunction(ILogicalExpression rec0, ILogicalExpression rec1,
+            SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> recordMergeFnArgs = new ArrayList<>();
         recordMergeFnArgs.add(new MutableObject<>(rec0));
         recordMergeFnArgs.add(new MutableObject<>(rec1));
         AbstractFunctionCallExpression recordMergeFn = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.RECORD_MERGE), recordMergeFnArgs);
+        recordMergeFn.setSourceLocation(sourceLoc);
         return recordMergeFn;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
index 5644a6c..97e2174 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
@@ -74,8 +74,8 @@ public class IntroduceDynamicTypeCastForExternalFunctionRule implements IAlgebra
         IAType inputRecordType;
         ARecordType requiredRecordType;
         for (int iter1 = 0; iter1 < funcCallExpr.getArguments().size(); iter1++) {
-            inputRecordType = (IAType) op.computeOutputTypeEnvironment(context)
-                    .getType(funcCallExpr.getArguments().get(iter1).getValue());
+            Mutable<ILogicalExpression> argExpr = funcCallExpr.getArguments().get(iter1);
+            inputRecordType = (IAType) op.computeOutputTypeEnvironment(context).getType(argExpr.getValue());
             if (!(((ExternalScalarFunctionInfo) funcCallExpr.getFunctionInfo()).getArgumenTypes()
                     .get(iter1) instanceof ARecordType)) {
                 continue;
@@ -92,14 +92,16 @@ public class IntroduceDynamicTypeCastForExternalFunctionRule implements IAlgebra
                 inputRecordType = ((AUnionType) inputRecordType).getActualType();
                 checkUnknown = true;
             }
-            boolean castFlag = !IntroduceDynamicTypeCastRule.compatible(requiredRecordType, inputRecordType);
+            boolean castFlag = !IntroduceDynamicTypeCastRule.compatible(requiredRecordType, inputRecordType,
+                    argExpr.getValue().getSourceLocation());
             if (castFlag || checkUnknown) {
                 AbstractFunctionCallExpression castFunc =
                         new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE));
-                castFunc.getArguments().add(funcCallExpr.getArguments().get(iter1));
+                castFunc.setSourceLocation(argExpr.getValue().getSourceLocation());
+                castFunc.getArguments().add(argExpr);
                 TypeCastUtils.setRequiredAndInputTypes(castFunc, requiredRecordType, inputRecordType);
                 funcCallExpr.getArguments().set(iter1, new MutableObject<>(castFunc));
-                changed = changed || true;
+                changed = true;
             }
         }
         return changed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index 2695337..a7ed0fd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
+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.metadata.declared.DataSource;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -52,6 +54,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeRe
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Dynamically cast a variable from its type to a specified required type, in a
@@ -164,7 +167,7 @@ public class IntroduceDynamicTypeCastRule implements IAlgebraicRewriteRule {
         }
 
         /** see whether the input record type needs to be casted */
-        boolean cast = !compatible(requiredRecordType, inputRecordType);
+        boolean cast = !compatible(requiredRecordType, inputRecordType, op.getSourceLocation());
 
         if (checkUnknown) {
             recordVar = addWrapperFunction(requiredRecordType, recordVar, op, context, BuiltinFunctions.CHECK_UNKNOWN);
@@ -197,6 +200,7 @@ public class IntroduceDynamicTypeCastRule implements IAlgebraicRewriteRule {
         for (int index = 0; index < opRefs.size(); index++) {
             Mutable<ILogicalOperator> opRef = opRefs.get(index);
             ILogicalOperator op = opRef.getValue();
+            SourceLocation sourceLoc = op.getSourceLocation();
 
             /** get produced vars */
             List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
@@ -209,13 +213,16 @@ public class IntroduceDynamicTypeCastRule implements IAlgebraicRewriteRule {
                     IAType actualType = (IAType) env.getVarType(var);
                     AbstractFunctionCallExpression cast =
                             new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fd));
-                    cast.getArguments()
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    cast.setSourceLocation(sourceLoc);
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                    varRef.setSourceLocation(sourceLoc);
+                    cast.getArguments().add(new MutableObject<ILogicalExpression>(varRef));
                     /** enforce the required record type */
                     TypeCastUtils.setRequiredAndInputTypes(cast, requiredRecordType, actualType);
                     LogicalVariable newAssignVar = context.newVar();
                     AssignOperator newAssignOperator =
                             new AssignOperator(newAssignVar, new MutableObject<ILogicalExpression>(cast));
+                    newAssignOperator.setSourceLocation(sourceLoc);
                     newAssignOperator.getInputs().add(new MutableObject<ILogicalOperator>(op));
                     opRef.setValue(newAssignOperator);
                     context.computeAndSetTypeEnvironmentForOperator(newAssignOperator);
@@ -240,15 +247,18 @@ public class IntroduceDynamicTypeCastRule implements IAlgebraicRewriteRule {
      *
      * @param reqType
      * @param inputType
+     * @param sourceLoc
      * @return true if compatible; false otherwise
      * @throws AlgebricksException
      */
-    public static boolean compatible(ARecordType reqType, IAType inputType) throws AlgebricksException {
+    public static boolean compatible(ARecordType reqType, IAType inputType, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (inputType.getTypeTag() == ATypeTag.ANY) {
             return false;
         }
         if (inputType.getTypeTag() != ATypeTag.OBJECT) {
-            throw new AlgebricksException("The input type " + inputType + " is not a valid record type!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "The input type " + inputType + " is not a valid record type!");
         }
         ARecordType inputRecType = (ARecordType) inputType;
         if (reqType.isOpen() != inputRecType.isOpen()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
index 0fa4d30..f4d8419 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -63,6 +63,7 @@ public class IntroduceMaterializationForInsertWithSelfScanRule implements IAlgeb
 
         if (sameDataset) {
             MaterializeOperator materializeOperator = new MaterializeOperator();
+            materializeOperator.setSourceLocation(op.getSourceLocation());
             MaterializePOperator materializePOperator = new MaterializePOperator(true);
             materializeOperator.setPhysicalOperator(materializePOperator);
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
index c41601b..fd6ff25 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
@@ -67,6 +67,7 @@ public class IntroduceRandomPartitioningFeedComputationRule implements IAlgebrai
         }
 
         ExchangeOperator exchangeOp = new ExchangeOperator();
+        exchangeOp.setSourceLocation(op.getSourceLocation());
         INodeDomain runtimeDomain = feedDataSource.getComputationNodeDomain();
 
         exchangeOp.setPhysicalOperator(new RandomPartitionExchangePOperator(runtimeDomain));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 6d53c13..7e2e4ea 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -27,6 +27,8 @@ import java.util.Map;
 import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+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.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceIndex;
@@ -73,6 +75,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDelete
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule matches the pattern:
@@ -115,6 +118,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                 primaryIndexModificationOp.getAdditionalNonFilteringExpressions();
         LogicalVariable newRecordVar;
         LogicalVariable newMetaVar = null;
+        SourceLocation sourceLoc = primaryIndexModificationOp.getSourceLocation();
 
         /**
          * inputOp is the assign operator which extracts primary keys from the input
@@ -125,7 +129,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
         newRecordVar = getRecordVar(context, inputOp, newRecordExpr, 0);
         if (newMetaExprs != null && !newMetaExprs.isEmpty()) {
             if (newMetaExprs.size() > 1) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Number of meta records can't be more than 1. Number of meta records found = "
                                 + newMetaExprs.size());
             }
@@ -146,7 +150,8 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
         String datasetName = datasetSource.getId().getDatasourceName();
         Dataset dataset = mp.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Unknown dataset " + datasetName + " in dataverse " + dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
@@ -156,7 +161,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = mp.findType(dataset.getItemTypeDataverseName(), itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.OBJECT) {
-            throw new AlgebricksException("Only record types can be indexed.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
         // meta type
@@ -195,8 +200,9 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     .getAdditionalFilteringExpressions()) {
                 filteringExpression.getValue().getUsedVariables(filteringVars);
                 for (LogicalVariable var : filteringVars) {
-                    filteringExpressions
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                    varRef.setSourceLocation(filteringExpression.getValue().getSourceLocation());
+                    filteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                 }
             }
         }
@@ -207,6 +213,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
             // Split the logical plan into "each secondary index update branch"
             // to replicate each <PK,OBJECT> pair.
             replicateOp = new ReplicateOperator(secondaryIndexTotalCnt);
+            replicateOp.setSourceLocation(sourceLoc);
             replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
             context.computeAndSetTypeEnvironmentForOperator(replicateOp);
@@ -270,10 +277,14 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                         secondaryKeyFields.get(i), secondaryKeyTypes.get(i).getTypeTag());
                 LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
                 secondaryKeyVars.add(skVar);
-                secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(skVar)));
+                VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
+                skVarRef.setSourceLocation(sourceLoc);
+                secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
                 if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
-                    beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId))));
+                    VariableReferenceExpression varRef =
+                            new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
+                    varRef.setSourceLocation(sourceLoc);
+                    beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                 }
             }
 
@@ -283,7 +294,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                 Mutable<ILogicalExpression> filterExpression =
                         (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
                                 : createFilterExpression(secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
-                                        index.isOverridingKeyFieldTypes());
+                                        index.isOverridingKeyFieldTypes(), sourceLoc);
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
 
                 // Introduce the TokenizeOperator only when doing bulk-load,
@@ -306,8 +317,9 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     List<Mutable<ILogicalExpression>> tokenizeKeyExprs = new ArrayList<>();
                     LogicalVariable tokenVar = context.newVar();
                     tokenizeKeyVars.add(tokenVar);
-                    tokenizeKeyExprs
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(tokenVar)));
+                    VariableReferenceExpression tokenVarRef = new VariableReferenceExpression(tokenVar);
+                    tokenVarRef.setSourceLocation(sourceLoc);
+                    tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(tokenVarRef));
 
                     // Check the field type of the secondary key.
                     IAType secondaryKeyType;
@@ -325,8 +337,9 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     if (isPartitioned) {
                         LogicalVariable lengthVar = context.newVar();
                         tokenizeKeyVars.add(lengthVar);
-                        tokenizeKeyExprs
-                                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lengthVar)));
+                        VariableReferenceExpression lengthVarRef = new VariableReferenceExpression(lengthVar);
+                        lengthVarRef.setSourceLocation(sourceLoc);
+                        tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(lengthVarRef));
                         varTypes.add(BuiltinType.SHORTWITHOUTTYPEINFO);
                     }
 
@@ -335,6 +348,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                             primaryIndexModificationOp.getPrimaryKeyExpressions(), secondaryExpressions,
                             tokenizeKeyVars, filterExpression, primaryIndexModificationOp.getOperation(),
                             primaryIndexModificationOp.isBulkload(), isPartitioned, varTypes);
+                    tokenUpdate.setSourceLocation(sourceLoc);
                     tokenUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
                     context.computeAndSetTypeEnvironmentForOperator(tokenUpdate);
                     replicateOutput = tokenUpdate;
@@ -343,6 +357,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                             primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
                             primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                     : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                    indexUpdate.setSourceLocation(sourceLoc);
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
                 } else {
@@ -353,15 +368,18 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                             primaryIndexModificationOp.isBulkload(),
                             primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                     : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                    indexUpdate.setSourceLocation(sourceLoc);
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     replicateOutput = indexUpdate;
                     // We add the necessary expressions for upsert
                     if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
                         indexUpdate.setBeforeOpSecondaryKeyExprs(beforeOpSecondaryExpressions);
                         if (filteringFields != null) {
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar());
+                            varRef.setSourceLocation(sourceLoc);
                             indexUpdate.setBeforeOpAdditionalFilteringExpression(
-                                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                                            primaryIndexModificationOp.getBeforeOpFilterVar())));
+                                    new MutableObject<ILogicalExpression>(varRef));
                         }
                     }
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
@@ -383,8 +401,11 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     keyVarList.add(keyVar);
                     AbstractFunctionCallExpression createMBR =
                             new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
-                    createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(secondaryKeyVars.get(0))));
+                    createMBR.setSourceLocation(sourceLoc);
+                    VariableReferenceExpression secondaryKeyVarRef =
+                            new VariableReferenceExpression(secondaryKeyVars.get(0));
+                    secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                    createMBR.getArguments().add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(dimension)))));
                     createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
@@ -393,18 +414,22 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                 }
                 secondaryExpressions.clear();
                 for (LogicalVariable secondaryKeyVar : keyVarList) {
-                    secondaryExpressions.add(
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
+                    VariableReferenceExpression secondaryKeyVarRef = new VariableReferenceExpression(secondaryKeyVar);
+                    secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                    secondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                 }
                 if (isPointMBR && isBulkload) {
                     //for PointMBR optimization: see SecondaryRTreeOperationsHelper.buildLoadingJobSpec() and
                     //createFieldPermutationForBulkLoadOp(int) for more details.
                     for (LogicalVariable secondaryKeyVar : keyVarList) {
-                        secondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(secondaryKeyVar)));
+                        VariableReferenceExpression secondaryKeyVarRef =
+                                new VariableReferenceExpression(secondaryKeyVar);
+                        secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                        secondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     }
                 }
                 AssignOperator assignCoordinates = new AssignOperator(keyVarList, keyExprList);
+                assignCoordinates.setSourceLocation(sourceLoc);
                 assignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
                 context.computeAndSetTypeEnvironmentForOperator(assignCoordinates);
                 replicateOutput = assignCoordinates;
@@ -420,6 +445,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                         originalKeyVarList.add(keyVar);
                         AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                                 FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
+                        createMBR.setSourceLocation(sourceLoc);
                         createMBR.getArguments().add(beforeOpSecondaryExpressions.get(0));
                         createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                                 new ConstantExpression(new AsterixConstantValue(new AInt32(dimension)))));
@@ -429,10 +455,13 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     }
                     beforeOpSecondaryExpressions.clear();
                     for (LogicalVariable secondaryKeyVar : originalKeyVarList) {
-                        beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(secondaryKeyVar)));
+                        VariableReferenceExpression secondaryKeyVarRef =
+                                new VariableReferenceExpression(secondaryKeyVar);
+                        secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                        beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     }
                     originalAssignCoordinates = new AssignOperator(originalKeyVarList, originalKeyExprList);
+                    originalAssignCoordinates.setSourceLocation(sourceLoc);
                     originalAssignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
                     context.computeAndSetTypeEnvironmentForOperator(originalAssignCoordinates);
                 } else {
@@ -440,7 +469,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     // nullable.
                     boolean forceFilter = keyPairType.second;
                     filterExpression = createFilterExpression(keyVarList,
-                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
+                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter, sourceLoc);
                 }
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
                 indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
@@ -448,12 +477,16 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                         primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
                         primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                 : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                indexUpdate.setSourceLocation(sourceLoc);
                 indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                 if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
                     // set before op secondary key expressions
                     if (filteringFields != null) {
-                        indexUpdate.setBeforeOpAdditionalFilteringExpression(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar())));
+                        VariableReferenceExpression varRef =
+                                new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar());
+                        varRef.setSourceLocation(sourceLoc);
+                        indexUpdate.setBeforeOpAdditionalFilteringExpression(
+                                new MutableObject<ILogicalExpression>(varRef));
                     }
                     // set filtering expressions
                     indexUpdate.setBeforeOpSecondaryKeyExprs(beforeOpSecondaryExpressions);
@@ -546,6 +579,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
             boolean afterOp) throws AlgebricksException {
         List<LogicalVariable> vars = new ArrayList<>();
         List<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
+        SourceLocation sourceLoc = currentTop.getSourceLocation();
         for (Index index : indexes) {
             if (index.isPrimaryIndex()) {
                 continue;
@@ -566,7 +600,8 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                         ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordVar : metaVar : recordVar;
                 LogicalVariable fieldVar = context.newVar();
                 // create record variable ref
-                Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(sourceVar));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVar);
+                varRef.setSourceLocation(sourceLoc);
                 IAType fieldType = sourceType.getSubFieldType(indexFieldId.fieldName);
                 AbstractFunctionCallExpression theFieldAccessFunc;
                 if (fieldType == null) {
@@ -574,11 +609,12 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     // make handling of records with incorrect value type for this field easier and cleaner
                     context.addNotToBeInlinedVar(fieldVar);
                     // create field access
-                    AbstractFunctionCallExpression fieldAccessFunc =
-                            getOpenOrNestedFieldAccessFunction(varRef, indexFieldId.fieldName);
+                    AbstractFunctionCallExpression fieldAccessFunc = getOpenOrNestedFieldAccessFunction(
+                            new MutableObject<>(varRef), indexFieldId.fieldName, sourceLoc);
                     // create cast
                     theFieldAccessFunc = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(
                             index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX));
+                    theFieldAccessFunc.setSourceLocation(sourceLoc);
                     // The first argument is the field
                     theFieldAccessFunc.getArguments().add(new MutableObject<ILogicalExpression>(fieldAccessFunc));
                     TypeCastUtils.setRequiredAndInputTypes(theFieldAccessFunc, skTypes.get(i), BuiltinType.ANY);
@@ -587,9 +623,10 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                     int pos = indexFieldId.fieldName.size() > 1 ? -1
                             : sourceType.getFieldIndex(indexFieldId.fieldName.get(0));
                     // Field not found --> This is either an open field or a nested field. it can't be accessed by index
-                    theFieldAccessFunc =
-                            (pos == -1) ? getOpenOrNestedFieldAccessFunction(varRef, indexFieldId.fieldName)
-                                    : getClosedFieldAccessFunction(varRef, pos);
+                    theFieldAccessFunc = (pos == -1)
+                            ? getOpenOrNestedFieldAccessFunction(new MutableObject<>(varRef), indexFieldId.fieldName,
+                                    sourceLoc)
+                            : getClosedFieldAccessFunction(new MutableObject<>(varRef), pos, sourceLoc);
                 }
                 vars.add(fieldVar);
                 exprs.add(new MutableObject<ILogicalExpression>(theFieldAccessFunc));
@@ -598,6 +635,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
         }
         // AssignOperator assigns secondary keys to their vars
         AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
+        castedFieldAssignOperator.setSourceLocation(sourceLoc);
         return introduceNewOp(context, currentTop, castedFieldAssignOperator, afterOp);
     }
 
@@ -618,15 +656,17 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
     }
 
     private static AbstractFunctionCallExpression getClosedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            int position) {
+            int position, SourceLocation sourceLoc) {
         Mutable<ILogicalExpression> indexRef =
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(position))));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                varRef, indexRef);
+        ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
+        fnExpr.setSourceLocation(sourceLoc);
+        return fnExpr;
     }
 
     private static AbstractFunctionCallExpression getOpenOrNestedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            List<String> fields) {
+            List<String> fields, SourceLocation sourceLoc) {
         ScalarFunctionCallExpression func;
         if (fields.size() > 1) {
             IAObject fieldList = stringListToAOrderedList(fields);
@@ -641,6 +681,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
             func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
                     varRef, fieldRef);
         }
+        func.setSourceLocation(sourceLoc);
         return func;
     }
 
@@ -657,7 +698,8 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
     }
 
     private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
-            IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, boolean forceFilter, SourceLocation sourceLoc)
+            throws AlgebricksException {
         List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
         // Add 'is not null' to all nullable secondary index keys as a filtering
         // condition.
@@ -666,12 +708,16 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
             if (!NonTaggedFormatUtil.isOptional(secondaryKeyType) && !forceFilter) {
                 continue;
             }
+            VariableReferenceExpression secondaryKeyVarRef = new VariableReferenceExpression(secondaryKeyVar);
+            secondaryKeyVarRef.setSourceLocation(sourceLoc);
             ScalarFunctionCallExpression isUnknownFuncExpr =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_UNKNOWN),
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
+                            new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
+            isUnknownFuncExpr.setSourceLocation(sourceLoc);
             ScalarFunctionCallExpression notFuncExpr =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
                             new MutableObject<ILogicalExpression>(isUnknownFuncExpr));
+            notFuncExpr.setSourceLocation(sourceLoc);
             filterExpressions.add(new MutableObject<ILogicalExpression>(notFuncExpr));
         }
         // No nullable secondary keys.
@@ -681,8 +727,10 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
         Mutable<ILogicalExpression> filterExpression;
         if (filterExpressions.size() > 1) {
             // Create a conjunctive condition.
-            filterExpression = new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), filterExpressions));
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), filterExpressions);
+            andExpr.setSourceLocation(sourceLoc);
+            filterExpression = new MutableObject<>(andExpr);
         } else {
             filterExpression = filterExpressions.get(0);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
index c5cb6ae..926df0e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
@@ -64,12 +64,14 @@ public class IntroduceTransactionCommitByAssignOpRule implements IAlgebraicRewri
         LogicalVariable v = context.newVar();
         AssignOperator assignOperator =
                 new AssignOperator(v, new MutableObject<ILogicalExpression>(selectOperator.getCondition().getValue()));
-
+        assignOperator.setSourceLocation(selectOperator.getSourceLocation());
         //set the input of the new assign-operator to the input of the select-operator.
         assignOperator.getInputs().add(childOfSelect);
 
         //set the result value of the assign-operator to the condition of the select-operator
-        selectOperator.getCondition().setValue(new VariableReferenceExpression(v));//scalarFunctionCallExpression);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(v);
+        varRef.setSourceLocation(selectOperator.getSourceLocation());
+        selectOperator.getCondition().setValue(varRef);//scalarFunctionCallExpression);
         selectOperator.getInputs().set(0, new MutableObject<ILogicalOperator>(assignOperator));
 
         context.computeAndSetTypeEnvironmentForOperator(assignOperator);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
index 561afb9..02d8f14 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
@@ -85,11 +85,13 @@ public class IntroduceUnnestForCollectionToSequenceRule implements IAlgebraicRew
         }
         /** change the assign operator to an unnest operator */
         LogicalVariable var = assign.getVariables().get(0);
+        UnnestingFunctionCallExpression scanCollExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(argExpr));
+        scanCollExpr.setSourceLocation(func.getSourceLocation());
         @SuppressWarnings("unchecked")
-        UnnestOperator unnest = new UnnestOperator(var,
-                new MutableObject<ILogicalExpression>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
-                        new MutableObject<ILogicalExpression>(argExpr))));
+        UnnestOperator unnest = new UnnestOperator(var, new MutableObject<ILogicalExpression>(scanCollExpr));
+        unnest.setSourceLocation(assign.getSourceLocation());
         unnest.getInputs().addAll(assign.getInputs());
         opRef.setValue(unnest);
         context.computeAndSetTypeEnvironmentForOperator(unnest);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
index 2fe6460..7f23775 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleS
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * In principle, an unnest operator produces a sequence of items from a collection.
@@ -104,25 +105,35 @@ public class ListifyUnnestingFunctionRule implements IAlgebraicRewriteRule {
             return false;
         }
 
+        SourceLocation sourceLoc = func.getSourceLocation();
+
         // Generates the listified collection in a subplan.
         SubplanOperator subplanOperator = new SubplanOperator();
+        subplanOperator.setSourceLocation(sourceLoc);
         // Creates a nested tuple source operator.
         NestedTupleSourceOperator ntsOperator = new NestedTupleSourceOperator(new MutableObject<>(subplanOperator));
-
+        ntsOperator.setSourceLocation(sourceLoc);
         // Unnests the dataset.
         LogicalVariable unnestVar = context.newVar();
-        ILogicalExpression unnestExpr = new UnnestingFunctionCallExpression(functionInfo, func.getArguments());
+        UnnestingFunctionCallExpression unnestExpr =
+                new UnnestingFunctionCallExpression(functionInfo, func.getArguments());
+        unnestExpr.setSourceLocation(sourceLoc);
         UnnestOperator unnestOperator = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
+        unnestOperator.setSourceLocation(sourceLoc);
         unnestOperator.getInputs().add(new MutableObject<>(ntsOperator));
 
         // Listify the dataset into one collection.
         LogicalVariable aggVar = context.newVar();
-        Mutable<ILogicalExpression> aggArgExprRef = new MutableObject<>(new VariableReferenceExpression(unnestVar));
-        ILogicalExpression aggExpr =
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> aggArgExprRef = new MutableObject<>(unnestVarRef);
+        AggregateFunctionCallExpression aggExpr =
                 new AggregateFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false,
                         new ArrayList<>(Collections.singletonList(aggArgExprRef)));
+        aggExpr.setSourceLocation(sourceLoc);
         AggregateOperator aggregateOperator = new AggregateOperator(new ArrayList<>(Collections.singletonList(aggVar)),
                 new ArrayList<>(Collections.singletonList(new MutableObject<>(aggExpr))));
+        aggregateOperator.setSourceLocation(sourceLoc);
         aggregateOperator.getInputs().add(new MutableObject<>(unnestOperator));
 
         // Adds the aggregate operator as the root of the subplan.
@@ -136,7 +147,9 @@ public class ListifyUnnestingFunctionRule implements IAlgebraicRewriteRule {
         // asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3
         subplanOperator.getInputs().add(op.getInputs().get(0));
         op.getInputs().set(0, new MutableObject<>(subplanOperator));
-        exprRef.setValue(new VariableReferenceExpression(aggVar));
+        VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+        aggVarRef.setSourceLocation(sourceLoc);
+        exprRef.setValue(aggVarRef);
 
         // Computes type environments for new operators.
         context.computeAndSetTypeEnvironmentForOperator(ntsOperator);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index d6581a2..32fa744 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -25,6 +25,8 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.asterix.algebra.base.OperatorAnnotation;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -140,6 +142,7 @@ public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
                     // create an assign
                     LogicalVariable v = context.newVar();
                     AssignOperator a2 = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+                    a2.setSourceLocation(expr.getSourceLocation());
                     pushFieldAssign(a2, topOp, context);
                     context.computeAndSetTypeEnvironmentForOperator(a2);
                     ILogicalExpression arg = f.getArguments().get(0).getValue();
@@ -154,7 +157,9 @@ public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
                             context.addPrimaryKey(pk);
                         }
                     }
-                    exprRef.setValue(new VariableReferenceExpression(v));
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(v);
+                    varRef.setSourceLocation(expr.getSourceLocation());
+                    exprRef.setValue(varRef);
                     return true;
                 } else {
                     boolean pushed = false;
@@ -217,7 +222,7 @@ public class LoadRecordFieldsRule implements IAlgebraicRewriteRule {
                         }
                     }
                 }
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, a2.getSourceLocation(),
                         "Field access " + getFirstExpr(a2) + " does not correspond to any input of operator " + topOp);
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index 13ff0ee..cd99c28 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -21,6 +21,8 @@ package org.apache.asterix.optimizer.rules;
 import java.util.ArrayList;
 import java.util.List;
 
+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.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.IMutationDataSource;
@@ -45,6 +47,7 @@ import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule rewrites all meta() function calls in a query plan
@@ -204,7 +207,8 @@ class LogicalExpressionReferenceTransform implements ILogicalExpressionReference
         }
         // The user query provides more than one parameter for the meta function.
         if (argRefs.size() > 1) {
-            throw new AlgebricksException("The meta function can at most have one argument!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                    "The meta function can at most have one argument!");
         }
 
         // The user query provides exact one parameter for the meta function.
@@ -218,16 +222,21 @@ class LogicalExpressionReferenceTransform implements ILogicalExpressionReference
             if (!dataVar.equals(argVar)) {
                 return false;
             }
-            exprRef.setValue(new VariableReferenceExpression(metaVar));
+            VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+            metaVarRef.setSourceLocation(expr.getSourceLocation());
+            exprRef.setValue(metaVarRef);
             return true;
         }
 
         // The user query provides zero parameter for the meta function.
         if (variableRequired) {
-            throw new AlgebricksException("Cannot resolve to ambiguity on the meta function call --"
-                    + " there are more than one dataset choices!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                    "Cannot resolve to ambiguity on the meta function call --"
+                            + " there are more than one dataset choices!");
         }
-        exprRef.setValue(new VariableReferenceExpression(metaVar));
+        VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+        metaVarRef.setSourceLocation(expr.getSourceLocation());
+        exprRef.setValue(metaVarRef);
         return true;
     }
 }
@@ -268,6 +277,7 @@ class MetaKeyToFieldAccessTransform implements ILogicalExpressionReferenceTransf
         if (!funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.META_KEY)) {
             return false;
         }
+        SourceLocation sourceLoc = expr.getSourceLocation();
         // Get arguments
         // first argument : Resource key
         // second argument: field
@@ -286,13 +296,18 @@ class MetaKeyToFieldAccessTransform implements ILogicalExpressionReferenceTransf
                 functionIdentifier = BuiltinFunctions.FIELD_ACCESS_BY_NAME;
                 break;
             default:
-                throw new AlgebricksException("Unsupported field name type " + fieldNameType.getTypeTag());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported field name type " + fieldNameType.getTypeTag());
         }
         IFunctionInfo finfoAccess = FunctionUtil.getFunctionInfo(functionIdentifier);
         ArrayList<Mutable<ILogicalExpression>> argExprs = new ArrayList<>(2);
-        argExprs.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
+        VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+        metaVarRef.setSourceLocation(sourceLoc);
+        argExprs.add(new MutableObject<>(metaVarRef));
         argExprs.add(new MutableObject<>(fieldNameExpression));
-        exprRef.setValue(new ScalarFunctionCallExpression(finfoAccess, argExprs));
+        ScalarFunctionCallExpression fAccessExpr = new ScalarFunctionCallExpression(finfoAccess, argExprs);
+        fAccessExpr.setSourceLocation(sourceLoc);
+        exprRef.setValue(fAccessExpr);
         return true;
     }
 }
@@ -321,7 +336,9 @@ class MetaKeyExpressionReferenceTransform implements ILogicalExpressionReference
         // Function is meta key access
         for (int i = 0; i < metaKeyAccessExpressions.size(); i++) {
             if (metaKeyAccessExpressions.get(i).equals(funcExpr)) {
-                exprRef.setValue(new VariableReferenceExpression(keyVars.get(i)));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(keyVars.get(i));
+                varRef.setSourceLocation(expr.getSourceLocation());
+                exprRef.setValue(varRef);
                 return true;
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
index 233171e..f51358b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
@@ -63,8 +63,10 @@ public class PullPositionalVariableFromUnnestRule implements IAlgebraicRewriteRu
         ArrayList<Mutable<ILogicalExpression>> rOpExprList = new ArrayList<Mutable<ILogicalExpression>>();
         StatefulFunctionCallExpression fce = new StatefulFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+        fce.setSourceLocation(op.getSourceLocation());
         rOpExprList.add(new MutableObject<ILogicalExpression>(fce));
         RunningAggregateOperator rOp = new RunningAggregateOperator(rOpVars, rOpExprList);
+        rOp.setSourceLocation(unnest.getSourceLocation());
         rOp.setExecutionMode(unnest.getExecutionMode());
         RunningAggregatePOperator rPop = new RunningAggregatePOperator();
         rOp.setPhysicalOperator(rPop);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
index cf5088b..82c0d4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -221,12 +221,15 @@ public class PushAggFuncIntoStandaloneAggregateRule implements IAlgebraicRewrite
             aggArgs.add(aggOpExpr.getArguments().get(0));
             AggregateFunctionCallExpression aggFuncExpr =
                     BuiltinFunctions.makeAggregateFunctionExpression(aggFuncIdent, aggArgs);
+            aggFuncExpr.setSourceLocation(assignFuncExpr.getSourceLocation());
             LogicalVariable newVar = context.newVar();
             aggOp.getVariables().add(newVar);
             aggOp.getExpressions().add(new MutableObject<ILogicalExpression>(aggFuncExpr));
 
             // The assign now just "renames" the variable to make sure the upstream plan still works.
-            srcAssignExprRef.setValue(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(assignFuncExpr.getSourceLocation());
+            srcAssignExprRef.setValue(newVarRef);
         }
 
         context.computeAndSetTypeEnvironmentForOperator(aggOp);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ee54cc02/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
index ccf7ccb..3c97ec8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
@@ -285,8 +285,10 @@ public class PushAggregateIntoNestedSubplanRule implements IAlgebraicRewriteRule
                                 LogicalVariable newVar = context.newVar();
                                 AggregateFunctionCallExpression aggFun =
                                         BuiltinFunctions.makeAggregateFunctionExpression(fi, fce.getArguments());
+                                aggFun.setSourceLocation(expr.getSourceLocation());
                                 rewriteAggregateInNestedSubplan(argVar, nspOp, aggFun, newVar, context);
-                                ILogicalExpression newVarExpr = new VariableReferenceExpression(newVar);
+                                VariableReferenceExpression newVarExpr = new VariableReferenceExpression(newVar);
+                                newVarExpr.setSourceLocation(expr.getSourceLocation());
                                 aggregateExprToVarExpr.put(expr, newVarExpr);
                                 return new Pair<>(Boolean.TRUE, newVarExpr);
                             } else {
@@ -328,6 +330,7 @@ public class PushAggregateIntoNestedSubplanRule implements IAlgebraicRewriteRule
                             (AbstractFunctionCallExpression) aggOp.getExpressions().get(i).getValue();
                     AggregateFunctionCallExpression newAggFun = BuiltinFunctions
                             .makeAggregateFunctionExpression(aggFun.getFunctionIdentifier(), new ArrayList<>());
+                    newAggFun.setSourceLocation(oldAggExpr.getSourceLocation());
                     for (Mutable<ILogicalExpression> arg : oldAggExpr.getArguments()) {
                         ILogicalExpression cloned = arg.getValue().cloneExpression();
                         newAggFun.getArguments().add(new MutableObject<>(cloned));