Merge branch 'release-0.9.4-pre-rc' into master

Change-Id: I5c0f14671f13c4415d1377f5044925032dd77cd1
diff --git a/asterixdb/asterix-active/pom.xml b/asterixdb/asterix-active/pom.xml
index a6f6a38..922a0e8 100644
--- a/asterixdb/asterix-active/pom.xml
+++ b/asterixdb/asterix-active/pom.xml
@@ -35,6 +35,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index bb76204..2aebc59 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -178,7 +178,7 @@
 
     private void stopIfRunning(ActiveRuntimeId runtimeId, IActiveRuntime runtime)
             throws HyracksDataException, InterruptedException {
-        if (runtimes.remove(runtimeId) != null) {
+        if (runtimes.containsKey(runtimeId)) {
             runtime.stop();
         } else {
             LOGGER.info("Not stopping already stopped runtime " + runtimeId);
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
index 1df9020..2f30df4 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
@@ -63,8 +63,10 @@
 
     @Override
     public final void stop() throws HyracksDataException, InterruptedException {
-        abort();
         synchronized (this) {
+            if (!done) {
+                abort();
+            }
             while (!done) {
                 wait();
             }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
deleted file mode 100644
index 6633810..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.active;
-
-public class CountRetryPolicy implements IRetryPolicy {
-
-    private final int count;
-    private int attempted = 0;
-
-    public CountRetryPolicy(int count) {
-        this.count = count;
-    }
-
-    @Override
-    public boolean retry() {
-        if (attempted < count) {
-            attempted++;
-            return true;
-        }
-        return false;
-    }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
index 5e26ae4..477bb5a 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.CountRetryPolicy;
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class CountRetryPolicyFactory implements IRetryPolicyFactory {
 
     private final int count;
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
deleted file mode 100644
index bc6f1b1..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.active;
-
-import java.time.Instant;
-import java.util.Date;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.common.transactions.ITxnIdFactory;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.DeployedJobSpecId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * Provides functionality for running DeployedJobSpecs
- */
-public class DeployedJobService {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    //To enable new Asterix TxnId for separate deployed job spec invocations
-    private static final byte[] TRANSACTION_ID_PARAMETER_NAME = "TxnIdParameter".getBytes();
-
-    //pool size one (only running one thread at a time)
-    private static final int POOL_SIZE = 1;
-
-    //Starts running a deployed job specification periodically with an interval of "duration" seconds
-    public static ScheduledExecutorService startRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId,
-            IHyracksClientConnection hcc, long duration, Map<byte[], byte[]> jobParameters, EntityId entityId,
-            ITxnIdFactory txnIdFactory) {
-        ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(POOL_SIZE);
-        scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    if (!runRepetitiveDeployedJobSpec(distributedId, hcc, jobParameters, duration, entityId,
-                            txnIdFactory)) {
-                        scheduledExecutorService.shutdown();
-                    }
-                } catch (Exception e) {
-                    LOGGER.log(Level.ERROR, "Job Failed to run for " + entityId.getExtensionName() + " "
-                            + entityId.getDataverse() + "." + entityId.getEntityName() + ".", e);
-                }
-            }
-        }, duration, duration, TimeUnit.MILLISECONDS);
-        return scheduledExecutorService;
-    }
-
-    public static boolean runRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
-            Map<byte[], byte[]> jobParameters, long duration, EntityId entityId, ITxnIdFactory txnIdFactory)
-            throws Exception {
-        long executionMilliseconds = runDeployedJobSpec(distributedId, hcc, jobParameters, entityId, txnIdFactory);
-        if (executionMilliseconds > duration && LOGGER.isErrorEnabled()) {
-            LOGGER.log(Level.ERROR,
-                    "Periodic job for " + entityId.getExtensionName() + " " + entityId.getDataverse() + "."
-                            + entityId.getEntityName() + " was unable to meet the required period of " + duration
-                            + " milliseconds. Actually took " + executionMilliseconds + " execution will shutdown"
-                            + new Date());
-            return false;
-        }
-        return true;
-    }
-
-    public synchronized static long runDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
-            Map<byte[], byte[]> jobParameters, EntityId entityId, ITxnIdFactory txnIdFactory) throws Exception {
-        JobId jobId;
-        long startTime = Instant.now().toEpochMilli();
-
-        //Add the Asterix Transaction Id to the map
-        jobParameters.put(TRANSACTION_ID_PARAMETER_NAME, String.valueOf(txnIdFactory.create().getId()).getBytes());
-        jobId = hcc.startJob(distributedId, jobParameters);
-
-        hcc.waitForCompletion(jobId);
-        long executionMilliseconds = Instant.now().toEpochMilli() - startTime;
-
-        LOGGER.log(Level.INFO,
-                "Deployed Job execution completed for " + entityId.getExtensionName() + " " + entityId.getDataverse()
-                        + "." + entityId.getEntityName() + ". Took " + executionMilliseconds + " milliseconds ");
-
-        return executionMilliseconds;
-
-    }
-
-    @Override
-    public String toString() {
-        return "DeployedJobSpecService";
-    }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
deleted file mode 100644
index a010984..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.active;
-
-@FunctionalInterface
-public interface IRetryPolicy {
-    /**
-     * @return true if one more attempt should be done
-     */
-    boolean retry();
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
index a946337..82b1d5e 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 @FunctionalInterface
 public interface IRetryPolicyFactory {
     /**
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
index 074f8f4..6f43c64 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class InfiniteRetryPolicy implements IRetryPolicy {
 
     private final IActiveEntityEventsListener listener;
@@ -27,7 +29,7 @@
     }
 
     @Override
-    public boolean retry() {
+    public boolean retry(Throwable failure) {
         synchronized (listener) {
             try {
                 listener.wait(5000); //NOSONAR this method is being called in a while loop
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
index b31d245..d33e1da 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class InfiniteRetryPolicyFactory implements IRetryPolicyFactory {
 
     @Override
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
index 1596c17..b8af32b 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
@@ -18,15 +18,17 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+import org.apache.hyracks.util.NoRetryPolicy;
+
 public class NoRetryPolicyFactory implements IRetryPolicyFactory {
     public static final NoRetryPolicyFactory INSTANCE = new NoRetryPolicyFactory();
-    private static final IRetryPolicy policy = () -> false;
 
     private NoRetryPolicyFactory() {
     }
 
     @Override
     public IRetryPolicy create(IActiveEntityEventsListener listener) {
-        return policy;
+        return NoRetryPolicy.INSTANCE;
     }
 }
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 6a6bb8d..f3fdfc8 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -238,5 +238,9 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
index 724b284..15267aa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.algebra.extension;
 
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
@@ -29,7 +29,7 @@
 /**
  * An interface that provides an extension mechanism to extend a language with additional statements
  */
-public abstract class ExtensionStatement implements Statement {
+public abstract class ExtensionStatement extends AbstractStatement {
 
     @Override
     public final Kind getKind() {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index beb47a8..94d3461 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -47,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -59,6 +60,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -120,16 +122,28 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
+        ITupleFilterFactory tupleFilterFactory = null;
+        long outputLimit = -1;
+        if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+            UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+            outputLimit = unnestMapOp.getOutputLimit();
+            if (unnestMapOp.getSelectCondition() != null) {
+                tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+                        typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+            }
+        }
         // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
         boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
                 jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
                 jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
-                unnestMap.getGenerateCallBackProceedResultVar());
+                tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar());
+        IOperatorDescriptor opDesc = btreeSearch.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
-        builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, btreeSearch.second);
 
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
index f06fe45..a24a83b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
@@ -85,6 +85,7 @@
 
         //get dataset splits
         IPushRuntimeFactory runtime = dataset.getCommitRuntimeFactory(metadataProvider, primaryKeyFields, isSink);
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 1d79455..fc899c6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -135,8 +135,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalLoopup =
                 metadataProvider.buildExternalDataLookupRuntime(builder.getJobSpec(), dataset, ridIndexes, retainInput,
                         typeEnv, opSchema, context, metadataProvider, retainMissing);
-        builder.contributeHyracksOperator(unnestMap, externalLoopup.first);
-        builder.contributeAlgebricksPartitionConstraint(externalLoopup.first, externalLoopup.second);
+        IOperatorDescriptor opDesc = externalLoopup.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, externalLoopup.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index eeb6688..7b5eb83 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -120,10 +120,12 @@
                         jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
                         jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
                         minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch(), frameLimit);
+        IOperatorDescriptor opDesc = invIndexSearch.first;
+        opDesc.setSourceLocation(unnestMapOp.getSourceLocation());
 
         // Contribute operator in hyracks job.
-        builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
+        builder.contributeHyracksOperator(unnestMapOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, invIndexSearch.second);
         ILogicalOperator srcExchange = unnestMapOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 949be23..a80e5b3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -102,9 +102,11 @@
                 builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
                 dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
                 maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
+        IOperatorDescriptor opDesc = rtreeSearch.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
-        builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, rtreeSearch.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 02d79c4..32f8001 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -194,10 +194,11 @@
         }
     }
 
-    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env,
+    private IFunctionDescriptor resolveFunction(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
             JobGenContext context) throws AlgebricksException {
-        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
+        FunctionIdentifier fnId = expr.getFunctionIdentifier();
         IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
+        fd.setSourceLocation(expr.getSourceLocation());
         IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
         if (fnTypeInfer != null) {
             CompilerProperties compilerProps = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index cd217ab..134c96f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -63,6 +63,7 @@
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
 import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
 import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
+import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
 import org.apache.asterix.optimizer.rules.PushProperJoinThroughProduct;
 import org.apache.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
 import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
@@ -350,6 +351,7 @@
         // We are going to apply a constant folding rule again for this case.
         physicalRewritesTopLevel.add(new ConstantFoldingRule(appCtx));
         physicalRewritesTopLevel.add(new PushLimitIntoOrderByRule());
+        physicalRewritesTopLevel.add(new PushLimitIntoPrimarySearchRule());
         physicalRewritesTopLevel.add(new IntroduceProjectsRule());
         physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
index b9b5340..8219bcd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
@@ -112,14 +112,16 @@
                 LogicalVariable fieldVar = varExpr.getVariableReference();
                 Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(assignOp);
                 if (ecs == null) {
-                    ecs = new HashMap<LogicalVariable, EquivalenceClass>();
+                    ecs = new HashMap<>();
                     context.putEquivalenceClassMap(assignOp, ecs);
                 }
-                ILogicalExpression expr = new ScalarFunctionCallExpression(
+                VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+                recordVarRef.setSourceLocation(funcExpr.getSourceLocation());
+                ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(recordVar)),
-                        new MutableObject<ILogicalExpression>(
+                        new MutableObject<>(recordVarRef), new MutableObject<>(
                                 new ConstantExpression(new AsterixConstantValue(new AInt32(parameterIndex / 2))))); // Every two parameters corresponds to a field.
+                expr.setSourceLocation(funcExpr.getSourceLocation());
                 EquivalenceClass equivClass = new EquivalenceClass(Collections.singletonList(fieldVar), fieldVar,
                         Collections.singletonList(expr));
                 ecs.put(fieldVar, equivClass);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
index 5a70d3c..8bfd120 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.rewriter.rules.AbstractIntroduceGroupByCombinerRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AsterixIntroduceGroupByCombinerRule extends AbstractIntroduceGroupByCombinerRule {
 
@@ -43,36 +44,46 @@
     @Override
     protected void processNullTest(IOptimizationContext context, GroupByOperator nestedGby,
             List<LogicalVariable> aggregateVarsProducedByCombiner) {
+        SourceLocation sourceLoc = nestedGby.getSourceLocation();
         IFunctionInfo finfoEq = context.getMetadataProvider().lookupFunction(BuiltinFunctions.IS_SYSTEM_NULL);
         SelectOperator selectNonSystemNull;
 
         if (aggregateVarsProducedByCombiner.size() == 1) {
-            ILogicalExpression isSystemNullTest =
-                    new ScalarFunctionCallExpression(finfoEq, new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(aggregateVarsProducedByCombiner.get(0))));
+            VariableReferenceExpression aggVarRef =
+                    new VariableReferenceExpression(aggregateVarsProducedByCombiner.get(0));
+            aggVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression isSystemNullTest =
+                    new ScalarFunctionCallExpression(finfoEq, new MutableObject<>(aggVarRef));
+            isSystemNullTest.setSourceLocation(sourceLoc);
             IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
             ScalarFunctionCallExpression nonSystemNullTest =
-                    new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isSystemNullTest));
-            selectNonSystemNull =
-                    new SelectOperator(new MutableObject<ILogicalExpression>(nonSystemNullTest), false, null);
+                    new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
+            nonSystemNullTest.setSourceLocation(sourceLoc);
+            selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest), false, null);
+            selectNonSystemNull.setSourceLocation(sourceLoc);
         } else {
-            List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<>();
             for (LogicalVariable aggVar : aggregateVarsProducedByCombiner) {
-                ILogicalExpression isSystemNullTest = new ScalarFunctionCallExpression(finfoEq,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(aggVar)));
+                VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+                aggVarRef.setSourceLocation(sourceLoc);
+                ScalarFunctionCallExpression isSystemNullTest =
+                        new ScalarFunctionCallExpression(finfoEq, new MutableObject<>(aggVarRef));
+                isSystemNullTest.setSourceLocation(sourceLoc);
                 IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
-                ScalarFunctionCallExpression nonSystemNullTest = new ScalarFunctionCallExpression(finfoNot,
-                        new MutableObject<ILogicalExpression>(isSystemNullTest));
-                isSystemNullTestList.add(new MutableObject<ILogicalExpression>(nonSystemNullTest));
+                ScalarFunctionCallExpression nonSystemNullTest =
+                        new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
+                nonSystemNullTest.setSourceLocation(sourceLoc);
+                isSystemNullTestList.add(new MutableObject<>(nonSystemNullTest));
             }
             IFunctionInfo finfoAnd = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
-            selectNonSystemNull = new SelectOperator(new MutableObject<ILogicalExpression>(
-                    new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+            selectNonSystemNull = new SelectOperator(
+                    new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+            selectNonSystemNull.setSourceLocation(sourceLoc);
         }
 
         //add the not-system-null check into the nested pipeline
         Mutable<ILogicalOperator> ntsBeforeNestedGby = nestedGby.getInputs().get(0);
-        nestedGby.getInputs().set(0, new MutableObject<ILogicalOperator>(selectNonSystemNull));
+        nestedGby.getInputs().set(0, new MutableObject<>(selectNonSystemNull));
         selectNonSystemNull.getInputs().add(ntsBeforeNestedGby);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
index 5cf6769..b0c3bdf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ByNameToByHandleFieldAccessRule implements IAlgebraicRewriteRule {
 
@@ -65,6 +66,7 @@
         AbstractFunctionCallExpression fce =
                 (AbstractFunctionCallExpression) fieldAccessOp.getExpressions().get(0).getValue();
         ILogicalExpression a1 = fce.getArguments().get(0).getValue();
+        SourceLocation sourceLoc = fieldAccessOp.getSourceLocation();
 
         VariableReferenceExpression x;
         if (a1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
@@ -76,7 +78,9 @@
             ArrayList<Mutable<ILogicalExpression>> exprArray = new ArrayList<Mutable<ILogicalExpression>>(1);
             exprArray.add(new MutableObject<ILogicalExpression>(a1));
             AssignOperator assignVar = new AssignOperator(varArray, exprArray);
+            assignVar.setSourceLocation(sourceLoc);
             x = new VariableReferenceExpression(var1);
+            x.setSourceLocation(sourceLoc);
             assignVar.getInputs().add(opUnder);
             opUnder = new MutableObject<ILogicalOperator>(assignVar);
         }
@@ -88,24 +92,33 @@
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.TYPE_OF));
         typeOf.getArguments().add(new MutableObject<ILogicalExpression>(x));
         AssignOperator typAssign = new AssignOperator(t, new MutableObject<ILogicalExpression>(typeOf));
+        typAssign.setSourceLocation(sourceLoc);
         typAssign.getInputs().add(opUnder);
 
         // let $w := get-handle($t, path-expression)
         LogicalVariable w = context.newVar();
         AbstractFunctionCallExpression getHandle =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_HANDLE));
-        getHandle.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(t)));
+        getHandle.setSourceLocation(sourceLoc);
+        VariableReferenceExpression tRef = new VariableReferenceExpression(t);
+        tRef.setSourceLocation(sourceLoc);
+        getHandle.getArguments().add(new MutableObject<ILogicalExpression>(tRef));
         // the accessed field
         getHandle.getArguments().add(new MutableObject<ILogicalExpression>(fce.getArguments().get(1).getValue()));
         AssignOperator handleAssign = new AssignOperator(w, new MutableObject<ILogicalExpression>(getHandle));
+        handleAssign.setSourceLocation(sourceLoc);
         handleAssign.getInputs().add(new MutableObject<ILogicalOperator>(typAssign));
 
         // let $y := get-data(x, $w)
         AbstractFunctionCallExpression getData =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_DATA));
+        getData.setSourceLocation(sourceLoc);
         VariableReferenceExpression ref2 = new VariableReferenceExpression(x.getVariableReference());
+        ref2.setSourceLocation(sourceLoc);
         getData.getArguments().add(new MutableObject<ILogicalExpression>(ref2));
-        getData.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(w)));
+        VariableReferenceExpression wRef = new VariableReferenceExpression(w);
+        wRef.setSourceLocation(sourceLoc);
+        getData.getArguments().add(new MutableObject<ILogicalExpression>(wRef));
         fieldAccessOp.getExpressions().get(0).setValue(getData);
         List<Mutable<ILogicalOperator>> faInputs = fieldAccessOp.getInputs();
         faInputs.clear();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
index 19cd4b6..9c478ee 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
@@ -47,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ByNameToByIndexFieldAccessRule implements IAlgebraicRewriteRule {
 
@@ -100,10 +101,14 @@
         if (firstArg.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
+        SourceLocation sourceLoc = firstArg.getSourceLocation();
         LogicalVariable var1 = context.newVar();
         AssignOperator assignOp = new AssignOperator(new ArrayList<>(Collections.singletonList(var1)),
                 new ArrayList<>(Collections.singletonList(new MutableObject<>(firstArg))));
-        fce.getArguments().get(0).setValue(new VariableReferenceExpression(var1));
+        assignOp.setSourceLocation(sourceLoc);
+        VariableReferenceExpression var1Ref = new VariableReferenceExpression(var1);
+        var1Ref.setSourceLocation(sourceLoc);
+        fce.getArguments().get(0).setValue(var1Ref);
         assignOp.getInputs().add(new MutableObject<>(op.getInputs().get(0).getValue()));
         op.getInputs().get(0).setValue(assignOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
@@ -148,8 +153,10 @@
         if (k < 0) {
             return null;
         }
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                fce.getArguments().get(0),
+        ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), fce.getArguments().get(0),
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(k)))));
+        faExpr.setSourceLocation(fce.getSourceLocation());
+        return faExpr;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
index 6cc87ad..567e6c3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 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;
 
 import com.google.common.collect.ImmutableSet;
 
@@ -225,8 +226,11 @@
         ArrayList<LogicalVariable> assgnVars = new ArrayList<LogicalVariable>(1);
         assgnVars.add(unnest1.getVariable());
         ArrayList<Mutable<ILogicalExpression>> assgnExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        assgnExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(paramVar)));
+        VariableReferenceExpression paramRef = new VariableReferenceExpression(paramVar);
+        paramRef.setSourceLocation(arg0.getSourceLocation());
+        assgnExprs.add(new MutableObject<ILogicalExpression>(paramRef));
         AssignOperator assign = new AssignOperator(assgnVars, assgnExprs);
+        assign.setSourceLocation(arg0.getSourceLocation());
 
         LogicalVariable posVar = unnest1.getPositionalVariable();
         if (posVar == null) {
@@ -252,6 +256,7 @@
             opRef.setValue(assign);
             assign.getInputs().add(aggInputOpRef);
             AssignOperator gbyKeyAssign = new AssignOperator(gbyKeyAssgnVars, gbyKeyAssgnExprs);
+            gbyKeyAssign.setSourceLocation(gby.getSourceLocation());
             gbyKeyAssign.getInputs().add(gby.getInputs().get(0));
             bottomOpRef.setValue(gbyKeyAssign);
 
@@ -266,6 +271,8 @@
             nestedAssignVars.add(unnest1.getVariable());
             nestedAssignExprs.add(new MutableObject<ILogicalExpression>(arg0));
             AssignOperator nestedAssign = new AssignOperator(nestedAssignVars, nestedAssignExprs);
+            SourceLocation sourceLoc = unnest1.getSourceLocation();
+            nestedAssign.setSourceLocation(sourceLoc);
             nestedAssign.getInputs().add(opRef2);
 
             // Then create running aggregation for the positional variable
@@ -274,8 +281,10 @@
             raggVars.add(posVar);
             StatefulFunctionCallExpression fce = new StatefulFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+            fce.setSourceLocation(sourceLoc);
             raggExprs.add(new MutableObject<ILogicalExpression>(fce));
             RunningAggregateOperator raggOp = new RunningAggregateOperator(raggVars, raggExprs);
+            raggOp.setSourceLocation(sourceLoc);
             raggOp.setExecutionMode(unnest1.getExecutionMode());
             RunningAggregatePOperator raggPOp = new RunningAggregatePOperator();
             raggOp.setPhysicalOperator(raggPOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
index ab359bf..1eb3e75 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
@@ -19,6 +19,8 @@
 
 package org.apache.asterix.optimizer.rules;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
@@ -62,7 +64,7 @@
         IAType condType = (IAType) env.getType(condition);
         if (condType.getTypeTag() != ATypeTag.BOOLEAN && condType.getTypeTag() != ATypeTag.ANY
                 && !isPossibleBoolean(condType)) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, condition.getSourceLocation(),
                     "The select condition " + condition.toString() + " should be of the boolean type.");
         }
         return false;
@@ -75,11 +77,13 @@
      * @return true if it is; false otherwise.
      */
     private boolean isPossibleBoolean(IAType type) {
-        while (NonTaggedFormatUtil.isOptional(type)) {
-            IAType actualType = ((AUnionType) type).getActualType();
+        IAType checkingType = type;
+        while (NonTaggedFormatUtil.isOptional(checkingType)) {
+            IAType actualType = ((AUnionType) checkingType).getActualType();
             if (actualType.getTypeTag() == ATypeTag.BOOLEAN || actualType.getTypeTag() == ATypeTag.ANY) {
                 return true;
             }
+            checkingType = actualType;
         }
         return false;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
index 7c01ebd..5849d88 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
@@ -42,8 +42,9 @@
         if (checked) {
             return false;
         }
-        if (InsertUpsertCheckUtil.check(opRef.getValue())) {
-            throw new CompilationException(ErrorCode.COMPILATION_INVALID_RETURNING_EXPRESSION);
+        ILogicalOperator op = opRef.getValue();
+        if (InsertUpsertCheckUtil.check(op)) {
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_RETURNING_EXPRESSION, op.getSourceLocation());
         }
         checked = true;
         return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 29a8e77..7110dbb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -37,6 +38,7 @@
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -78,6 +80,7 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 
 public class ConstantFoldingRule implements IAlgebraicRewriteRule {
@@ -95,6 +98,9 @@
             BuiltinFunctions.META_KEY, BuiltinFunctions.RECORD_CONCAT, BuiltinFunctions.RECORD_CONCAT_STRICT,
             BuiltinFunctions.TO_ATOMIC, BuiltinFunctions.TO_ARRAY);
 
+    private static final Map<FunctionIdentifier, IAObject> FUNC_ID_TO_CONSTANT = ImmutableMap
+            .of(BuiltinFunctions.NUMERIC_E, new ADouble(Math.E), BuiltinFunctions.NUMERIC_PI, new ADouble(Math.PI));
+
     /**
      * Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method.
      */
@@ -220,6 +226,10 @@
                         return new Pair<>(changed, expr);
                     }
                 }
+                IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
+                if (c != null) {
+                    return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(c)));
+                }
 
                 IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
                         _emptyTypeEnv, _emptySchemas, jobGenCtx);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
index 85ee0ef..952c12b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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;
 
 public class DisjunctivePredicateToJoinRule implements IAlgebraicRewriteRule {
 
@@ -117,6 +118,8 @@
             }
         }
 
+        SourceLocation sourceLoc = select.getSourceLocation();
+
         AOrderedList list = new AOrderedList(new AOrderedListType(valType, "orderedlist"));
         for (AsterixConstantValue value : values) {
             list.add(value.getObject());
@@ -129,14 +132,19 @@
         Mutable<ILogicalExpression> mutCExp = new MutableObject<>(cExp);
         IFunctionInfo scanFctInfo = BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.SCAN_COLLECTION);
         UnnestingFunctionCallExpression scanExp = new UnnestingFunctionCallExpression(scanFctInfo, mutCExp);
+        scanExp.setSourceLocation(sourceLoc);
         LogicalVariable scanVar = context.newVar();
         UnnestOperator unn = new UnnestOperator(scanVar, new MutableObject<>(scanExp));
+        unn.setSourceLocation(sourceLoc);
         unn.getInputs().add(new MutableObject<>(ets));
         context.computeAndSetTypeEnvironmentForOperator(unn);
 
         IFunctionInfo eqFctInfo = BuiltinFunctions.getAsterixFunctionInfo(AlgebricksBuiltinFunctions.EQ);
         AbstractFunctionCallExpression eqExp = new ScalarFunctionCallExpression(eqFctInfo);
-        eqExp.getArguments().add(new MutableObject<>(new VariableReferenceExpression(scanVar)));
+        eqExp.setSourceLocation(sourceLoc);
+        VariableReferenceExpression scanVarRef = new VariableReferenceExpression(scanVar);
+        scanVarRef.setSourceLocation(sourceLoc);
+        eqExp.getArguments().add(new MutableObject<>(scanVarRef));
         eqExp.getArguments().add(new MutableObject<>(varEx.cloneExpression()));
         eqExp.getAnnotations().put(IndexedNLJoinExpressionAnnotation.INSTANCE,
                 IndexedNLJoinExpressionAnnotation.INSTANCE);
@@ -145,6 +153,7 @@
         eqExp.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
 
         InnerJoinOperator jOp = new InnerJoinOperator(new MutableObject<>(eqExp));
+        jOp.setSourceLocation(sourceLoc);
         jOp.getInputs().add(new MutableObject<>(unn));
         jOp.getInputs().add(select.getInputs().get(0));
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
index 01b37a4..f340898 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
@@ -69,7 +69,8 @@
                 continue;
             }
             LogicalVariable v = extractExprIntoAssignOpRef(expr.getValue(), opRef2, context);
-            ILogicalExpression newExpr = new VariableReferenceExpression(v);
+            VariableReferenceExpression newExpr = new VariableReferenceExpression(v);
+            newExpr.setSourceLocation(expr.getValue().getSourceLocation());
             newExprList.add(new MutableObject<ILogicalExpression>(newExpr));
             changed = true;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
index 4fbd632..585077f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
@@ -65,7 +65,9 @@
             ILogicalExpression expr = orderPair.second.getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE && !AnalysisUtil.isAccessToFieldRecord(expr)) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                orderPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                orderPair.second.setValue(vRef);
             }
         }
         context.computeAndSetTypeEnvironmentForOperator(oo);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
index fef2323..6b03d32 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
@@ -63,9 +63,10 @@
         UnnestOperator unnest = (UnnestOperator) op;
         ILogicalExpression unnestExpr = unnest.getExpressionRef().getValue();
         if (needsScanCollection(unnestExpr, op)) {
-            ILogicalExpression newExpr =
+            UnnestingFunctionCallExpression newExpr =
                     new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
                             new MutableObject<ILogicalExpression>(unnestExpr));
+            newExpr.setSourceLocation(unnestExpr.getSourceLocation());
             unnest.getExpressionRef().setValue(newExpr);
             context.addToDontApplySet(this, op);
             return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
index c594ff8..bd79a38 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableInt;
@@ -102,7 +103,8 @@
                         && ((DelegateOperator) op).getDelegate() instanceof CommitOperator)) {
             for (Map.Entry<AbstractReplicateOperator, MutableInt> entry : replicateOperators.entrySet()) {
                 if (entry.getKey().getOutputs().size() != entry.getValue().getValue()) {
-                    throw new AlgebricksException(ErrorCode.ASTERIX, ErrorCode.COMPILATION_FAILED_DUE_TO_REPLICATE_OP);
+                    throw new CompilationException(ErrorCode.COMPILATION_FAILED_DUE_TO_REPLICATE_OP,
+                            op.getSourceLocation());
                 }
             }
             return false;
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.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 @@
 
                 // 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 @@
             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 @@
                     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 @@
             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 @@
                 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 @@
                         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 @@
             }
         }
 
-        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);
             }
         }
 
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.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 @@
         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 @@
             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;
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 @@
             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;
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 @@
                 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;
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.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 @@
         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 @@
             // 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 @@
         }
         // 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);
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 c3e036e..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.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 {
 
@@ -66,11 +67,14 @@
             throws AlgebricksException {
 
         // match: commit OR distribute-result OR SINK - ... followed by:
-        // [insert to internal dataset with autogenerated id] - assign - project
-        // produce: insert - assign - assign* - project
+        // [insert to internal dataset with autogenerated id] - assign - assign? - project
+        // produce: insert - assign - assign? - assign* - project
         // **
-        // OR [insert to internal dataset with autogenerated id] - assign - [datasource scan]
-        // produce insert - assign - assign* - datasource scan
+        // OR [insert to internal dataset with autogenerated id] - assign - assign? - [datasource scan]
+        // produce: insert - assign - assign? - assign* - datasource scan
+        // **
+        // where assign* is the newly added assign that adds the autogenerated id
+        // and assign? is an assign that may exist when a filter is used
 
         AbstractLogicalOperator currentOp = (AbstractLogicalOperator) opRef.getValue();
         if (currentOp.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR) {
@@ -119,36 +123,62 @@
         }
         AssignOperator assignOp = (AssignOperator) parentOp;
         LogicalVariable inputRecord;
+        SourceLocation inputRecordSourceLoc;
 
-        //TODO: bug here. will not work for internal datasets with filters since the pattern becomes 
-        //[project-assign-assign-insert]
+        boolean hasFilter = false;
         AbstractLogicalOperator grandparentOp = (AbstractLogicalOperator) parentOp.getInputs().get(0).getValue();
+        AbstractLogicalOperator newAssignParentOp = grandparentOp;
+        AbstractLogicalOperator newAssignChildOp = assignOp;
         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();
+            if (greatgrandparentOp.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+                return false;
+            }
+            //filter case
+            ProjectOperator projectOp = (ProjectOperator) greatgrandparentOp;
+            inputRecord = projectOp.getVariables().get(0);
+            inputRecordSourceLoc = projectOp.getSourceLocation();
+            newAssignParentOp = greatgrandparentOp;
+            newAssignChildOp = grandparentOp;
+            hasFilter = true;
         } else {
             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.getInputs().add(new MutableObject<ILogicalOperator>(grandparentOp));
-        assignOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+        newAssign.setSourceLocation(insertOpSourceLoc);
+        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(newAssignParentOp));
+        newAssignChildOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+        if (hasFilter) {
+            VariableUtilities.substituteVariables(newAssignChildOp, inputRecord, v, context);
+        }
         VariableUtilities.substituteVariables(assignOp, inputRecord, v, context);
         VariableUtilities.substituteVariables(insertOp, inputRecord, v, context);
         context.computeAndSetTypeEnvironmentForOperator(newAssign);
+        if (hasFilter) {
+            context.computeAndSetTypeEnvironmentForOperator(newAssignChildOp);
+        }
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
-        context.computeAndSetTypeEnvironmentForOperator(insertOp);;
+        context.computeAndSetTypeEnvironmentForOperator(insertOp);
         for (AbstractLogicalOperator op : opStack) {
             VariableUtilities.substituteVariables(op, inputRecord, v, context);
             context.computeAndSetTypeEnvironmentForOperator(op);
@@ -162,21 +192,25 @@
         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--) {
@@ -187,17 +221,20 @@
             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;
     }
 }
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 @@
         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 @@
                 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;
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.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.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 @@
         }
 
         /** 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 @@
         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 @@
                     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 @@
      *
      * @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()) {
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 @@
 
         if (sameDataset) {
             MaterializeOperator materializeOperator = new MaterializeOperator();
+            materializeOperator.setSourceLocation(op.getSourceLocation());
             MaterializePOperator materializePOperator = new MaterializePOperator(true);
             materializeOperator.setPhysicalOperator(materializePOperator);
 
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 @@
         }
 
         ExchangeOperator exchangeOp = new ExchangeOperator();
+        exchangeOp.setSourceLocation(op.getSourceLocation());
         INodeDomain runtimeDomain = feedDataSource.getComputationNodeDomain();
 
         exchangeOp.setPhysicalOperator(new RandomPartitionExchangePOperator(runtimeDomain));
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..ae74832 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 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.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 @@
                 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 @@
         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 @@
         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.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                    dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
@@ -156,7 +161,7 @@
         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 @@
                     .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 @@
             // 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 @@
                         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 @@
                 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 @@
                     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 @@
                     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 @@
                             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 @@
                             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 @@
                             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 @@
                     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 @@
                 }
                 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 @@
                         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 @@
                     }
                     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 @@
                     // 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 @@
                         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 @@
             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 @@
                         ? 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 @@
                     // 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 @@
                     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 @@
         }
         // 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 @@
     }
 
     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 @@
             func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
                     varRef, fieldRef);
         }
+        func.setSourceLocation(sourceLoc);
         return func;
     }
 
@@ -657,7 +698,8 @@
     }
 
     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 @@
             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 @@
         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);
         }
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 @@
         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);
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 @@
         }
         /** 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);
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.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 @@
             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 @@
         // 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);
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.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 @@
                     // 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 @@
                             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 @@
                         }
                     }
                 }
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, a2.getSourceLocation(),
                         "Field access " + getFirstExpr(a2) + " does not correspond to any input of operator " + topOp);
             }
         }
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 @@
 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.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 @@
         }
         // 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 @@
             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 @@
         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 @@
                 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 @@
         // 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;
             }
         }
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 @@
         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);
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 @@
             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);
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 @@
                                 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 @@
                             (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));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index 769894d..b779c83 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -25,6 +25,8 @@
 
 import org.apache.asterix.algebra.base.OperatorAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -125,7 +127,8 @@
 
         Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
         if (dataset == null) {
-            throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
+                    asid.getDatasourceName(), asid.getDataverseName());
         }
         if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             return false;
@@ -228,8 +231,10 @@
                 for (Pair<LogicalVariable, LogicalVariable> m : varMappings) {
                     LogicalVariable v2 = context.newVar();
                     LogicalVariable oldVar = access.getVariables().get(0);
+                    VariableReferenceExpression v2Ref = new VariableReferenceExpression(v2);
+                    v2Ref.setSourceLocation(g.getSourceLocation());
                     g.getDecorList().add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(oldVar,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v2))));
+                            new MutableObject<ILogicalExpression>(v2Ref)));
                     changed = true;
                     access.getVariables().set(0, v2);
                     VariableUtilities.substituteVariables(access, m.first, m.second, context);
@@ -275,8 +280,9 @@
                     }
                 }
             }
-            throw new AlgebricksException("Field access " + access.getExpressions().get(0).getValue()
-                    + " does not correspond to any input of operator " + op2);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, access.getSourceLocation(),
+                    "Field access " + access.getExpressions().get(0).getValue()
+                            + " does not correspond to any input of operator " + op2);
         } else {
             // Check if the accessed field is not one of the partitioning key
             // fields. If yes, we can equate the two variables.
@@ -294,7 +300,8 @@
                 MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
                 Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
                 if (dataset == null) {
-                    throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
+                            asid.getDatasourceName(), asid.getDataverseName());
                 }
                 if (dataset.getDatasetType() != DatasetType.INTERNAL) {
                     setAsFinal(access, context, finalAnnot);
@@ -319,7 +326,9 @@
                             return false;
                         }
                         LogicalVariable keyVar = scan.getVariables().get(p);
-                        access.getExpressions().get(0).setValue(new VariableReferenceExpression(keyVar));
+                        VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                        keyVarRef.setSourceLocation(varRef.getSourceLocation());
+                        access.getExpressions().get(0).setValue(keyVarRef);
                         return true;
 
                     }
@@ -368,8 +377,11 @@
             return false;
         }
         AssignOperator a2 = (AssignOperator) op2;
-        if (getFirstExpr(access).equals(getFirstExpr(a2))) {
-            access.getExpressions().get(0).setValue(new VariableReferenceExpression(a2.getVariables().get(0)));
+        ILogicalExpression accessExpr0 = getFirstExpr(access);
+        if (accessExpr0.equals(getFirstExpr(a2))) {
+            VariableReferenceExpression varRef = new VariableReferenceExpression(a2.getVariables().get(0));
+            varRef.setSourceLocation(accessExpr0.getSourceLocation());
+            access.getExpressions().get(0).setValue(varRef);
             return true;
         } else {
             return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
index 4c0247f..56dc675 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
@@ -133,6 +133,7 @@
 
         // Create the new ORDER operator, set the topK value, and replace the current one.
         OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), (int) topK);
+        newOrderOp.setSourceLocation(orderOp.getSourceLocation());
         newOrderOp.setPhysicalOperator(
                 new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort(), newOrderOp.getTopK()));
         newOrderOp.getInputs().addAll(orderOp.getInputs());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
new file mode 100644
index 0000000..921d231
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pattern:
+ * SCAN or UNNEST_MAP -> (SELECT)? -> (EXCHANGE)? -> LIMIT
+ * We push both SELECT condition and LIMIT to SCAN or UNNEST_MAP
+ *
+ */
+public class PushLimitIntoPrimarySearchRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op = opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
+            return false;
+        }
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        context.addToDontApplySet(this, op);
+
+        Long outputLimit = getOutputLimit((LimitOperator) op);
+        if (outputLimit == null) {
+            // we cannot push if limit is not constant
+            return false;
+        }
+
+        Mutable<ILogicalOperator> childOp = op.getInputs().get(0);
+        if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+            childOp = childOp.getValue().getInputs().get(0);
+        }
+        boolean changed = false;
+        if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.SELECT) {
+            changed = rewriteSelect(childOp, outputLimit);
+        } else {
+            changed = setLimitForScanOrUnnestMap(childOp.getValue(), outputLimit);
+        }
+        if (changed) {
+            OperatorPropertiesUtil.typeOpRec(opRef, context);
+        }
+        return changed;
+    }
+
+    private Long getOutputLimit(LimitOperator limit) {
+        if (limit.getMaxObjects().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return null;
+        }
+        long outputLimit = AccessMethodUtils.getInt64Constant(limit.getMaxObjects());
+        if (limit.getOffset() != null && limit.getOffset().getValue() != null) {
+            if (limit.getOffset().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return null;
+            }
+            outputLimit += AccessMethodUtils.getInt64Constant(limit.getOffset());
+        }
+        return outputLimit;
+    }
+
+    private boolean rewriteSelect(Mutable<ILogicalOperator> op, long outputLimit) throws AlgebricksException {
+        SelectOperator select = (SelectOperator) op.getValue();
+        Set<LogicalVariable> selectedVariables = new HashSet<>();
+        select.getCondition().getValue().getUsedVariables(selectedVariables);
+        ILogicalOperator child = select.getInputs().get(0).getValue();
+        boolean changed = false;
+        if (child.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+            DataSourceScanOperator scan = (DataSourceScanOperator) child;
+            if (isScanPushable(scan, selectedVariables)) {
+                scan.setSelectCondition(select.getCondition());
+                scan.setOutputLimit(outputLimit);
+                changed = true;
+            }
+        } else if (child.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+            UnnestMapOperator unnestMap = (UnnestMapOperator) child;
+            if (isUnnestMapPushable(unnestMap, selectedVariables)) {
+                unnestMap.setSelectCondition(select.getCondition());
+                unnestMap.setOutputLimit(outputLimit);
+                changed = true;
+            }
+        }
+        if (changed) {
+            // SELECT is not needed
+            op.setValue(child);
+        }
+        return changed;
+    }
+
+    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, long outputLimit) throws AlgebricksException {
+        if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+            DataSourceScanOperator scan = (DataSourceScanOperator) op;
+            if (isScanPushable(scan, Collections.emptySet())) {
+                scan.setOutputLimit(outputLimit);
+                return true;
+            }
+        } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+            UnnestMapOperator unnestMap = (UnnestMapOperator) op;
+            if (isUnnestMapPushable(unnestMap, Collections.emptySet())) {
+                unnestMap.setOutputLimit(outputLimit);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean isUnnestMapPushable(UnnestMapOperator op, Set<LogicalVariable> selectedVariables) {
+        if (op.getOutputLimit() >= 0) {
+            // already pushed
+            return false;
+        }
+        ILogicalExpression unnestExpr = op.getExpressionRef().getValue();
+        if (op.propagatesInput() || unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+        if (!f.getFunctionIdentifier().equals(BuiltinFunctions.INDEX_SEARCH)) {
+            return false;
+        }
+        AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+        jobGenParams.readFromFuncArgs(f.getArguments());
+        if (!jobGenParams.isPrimaryIndex()) {
+            return false;
+        }
+        if (!op.getScanVariables().containsAll(selectedVariables)) {
+            return false;
+        }
+        return true;
+    }
+
+    private boolean isScanPushable(DataSourceScanOperator op, Set<LogicalVariable> selectedVariables) {
+        if (op.getOutputLimit() >= 0) {
+            return false;
+        }
+        if (!op.getInputs().isEmpty()
+                && op.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            return false;
+        }
+        if (((DataSource) op.getDataSource()).getDatasourceType() != DataSource.Type.INTERNAL_DATASET) {
+            return false;
+        }
+        if (!op.getScanVariables().containsAll(selectedVariables)) {
+            return false;
+        }
+        return true;
+    }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 1d7a61a..364816b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -262,13 +262,16 @@
         List<LogicalVariable> lhs = new ArrayList<>();
         List<Mutable<ILogicalExpression>> rhs = new ArrayList<>();
         lhs.add(outerUnnest.getVariable());
-        rhs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(listifyVar)));
+        VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar);
+        listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation());
+        rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef));
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList();
         for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) {
             lhs.add(gbyPair.first);
             rhs.add(gbyPair.second);
         }
         AssignOperator assignOp = new AssignOperator(lhs, rhs);
+        assignOp.setSourceLocation(outerUnnest.getSourceLocation());
         assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator));
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         opRef.setValue(assignOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
index 7b46b39..96b052b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
@@ -205,8 +205,11 @@
         List<LogicalVariable> assgnVars = new ArrayList<>(1);
         assgnVars.add(unnest1.getVariable());
         List<Mutable<ILogicalExpression>> assgnExprs = new ArrayList<>(1);
-        assgnExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(paramVar)));
+        VariableReferenceExpression paramVarRef = new VariableReferenceExpression(paramVar);
+        paramVarRef.setSourceLocation(arg0.getSourceLocation());
+        assgnExprs.add(new MutableObject<ILogicalExpression>(paramVarRef));
         AssignOperator assign = new AssignOperator(assgnVars, assgnExprs);
+        assign.setSourceLocation(agg.getSourceLocation());
         assign.getInputs().add(agg.getInputs().get(0));
         context.computeAndSetTypeEnvironmentForOperator(assign);
         LogicalVariable posVar = unnest1.getPositionalVariable();
@@ -220,8 +223,10 @@
             List<Mutable<ILogicalExpression>> rAggExprs = new ArrayList<>(1);
             StatefulFunctionCallExpression tidFun = new StatefulFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+            tidFun.setSourceLocation(agg.getSourceLocation());
             rAggExprs.add(new MutableObject<ILogicalExpression>(tidFun));
             RunningAggregateOperator rAgg = new RunningAggregateOperator(raggVars, rAggExprs);
+            rAgg.setSourceLocation(agg.getSourceLocation());
             rAgg.getInputs().add(new MutableObject<ILogicalOperator>(assign));
             aggregateParentRef.setValue(rAgg);
             context.computeAndSetTypeEnvironmentForOperator(rAgg);
@@ -289,6 +294,7 @@
         List<LogicalVariable> assgnVars = new ArrayList<>(1);
         assgnVars.add(aggVar);
         AssignOperator assign = new AssignOperator(assgnVars, scanFunc.getArguments());
+        assign.setSourceLocation(agg.getSourceLocation());
         assign.getInputs().add(unnest.getInputs().get(0));
         context.computeAndSetTypeEnvironmentForOperator(assign);
         opRef.setValue(assign);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 4afccb0..fed17bd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -25,6 +25,8 @@
 import org.apache.asterix.algebra.operators.physical.InvertedIndexPOperator;
 import org.apache.asterix.algebra.operators.physical.RTreeSearchPOperator;
 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.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -131,6 +133,7 @@
                                     AggregateFunctionCallExpression serialAggExpr =
                                             BuiltinFunctions.makeSerializableAggregateFunctionExpression(
                                                     expr.getFunctionIdentifier(), expr.getArguments());
+                                    serialAggExpr.setSourceLocation(expr.getSourceLocation());
                                     if (mergeAggregationExpressionFactory.createMergeAggregation(
                                             originalVariables.get(i), serialAggExpr, context) == null) {
                                         hasIntermediateAgg = false;
@@ -157,6 +160,7 @@
                                         AggregateFunctionCallExpression serialAggExpr =
                                                 BuiltinFunctions.makeSerializableAggregateFunctionExpression(
                                                         expr.getFunctionIdentifier(), expr.getArguments());
+                                        serialAggExpr.setSourceLocation(expr.getSourceLocation());
                                         aggOp.getExpressions().get(i).setValue(serialAggExpr);
                                     }
                                     ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
@@ -184,8 +188,7 @@
                                         context.getPhysicalOptimizationConfig().getMaxFramesForGroupBy()));
                             }
                         }
-                    } else if (((AbstractLogicalOperator) (r0.getValue())).getOperatorTag()
-                            .equals(LogicalOperatorTag.RUNNINGAGGREGATE)) {
+                    } else if (r0.getValue().getOperatorTag().equals(LogicalOperatorTag.RUNNINGAGGREGATE)) {
                         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gby.getGroupByList();
                         List<LogicalVariable> columnList = new ArrayList<LogicalVariable>(gbyList.size());
                         for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gbyList) {
@@ -198,8 +201,9 @@
                         op.setPhysicalOperator(new PreclusteredGroupByPOperator(columnList, gby.isGroupAll(),
                                 context.getPhysicalOptimizationConfig().getMaxFramesForGroupBy()));
                     } else {
-                        throw new AlgebricksException("Unsupported nested operator within a group-by: "
-                                + ((AbstractLogicalOperator) (r0.getValue())).getOperatorTag().name());
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
+                                "Unsupported nested operator within a group-by: "
+                                        + r0.getValue().getOperatorTag().name());
                     }
                 }
             }
@@ -235,8 +239,9 @@
                                 mp.findDataSourceIndex(jobGenParams.getIndexName(), dataSourceId);
                         INodeDomain storageDomain = mp.findNodeDomain(dataset.getNodeGroupName());
                         if (dsi == null) {
-                            throw new AlgebricksException("Could not find index " + jobGenParams.getIndexName()
-                                    + " for dataset " + dataSourceId);
+                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                                    "Could not find index " + jobGenParams.getIndexName() + " for dataset "
+                                            + dataSourceId);
                         }
                         IndexType indexType = jobGenParams.getIndexType();
                         boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
@@ -288,13 +293,13 @@
     private static void generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
             throws AlgebricksException {
         if (gby.getNestedPlans().size() != 1) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
         }
         ILogicalPlan p0 = gby.getNestedPlans().get(0);
         if (p0.getRoots().size() != 1) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
         }
@@ -303,8 +308,9 @@
         Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
         AbstractLogicalOperator r0Logical = (AbstractLogicalOperator) r0.getValue();
         if (r0Logical.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
-            throw new AlgebricksException("The merge aggregation expression generation should not process a "
-                    + r0Logical.getOperatorTag() + " operator.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
+                    "The merge aggregation expression generation should not process a " + r0Logical.getOperatorTag()
+                            + " operator.");
         }
         AggregateOperator aggOp = (AggregateOperator) r0.getValue();
         List<Mutable<ILogicalExpression>> aggFuncRefs = aggOp.getExpressions();
@@ -312,11 +318,13 @@
         int n = aggOp.getExpressions().size();
         List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
         for (int i = 0; i < n; i++) {
+            ILogicalExpression aggFuncExpr = aggFuncRefs.get(i).getValue();
             ILogicalExpression mergeExpr = mergeAggregationExpressionFactory
-                    .createMergeAggregation(aggProducedVars.get(i), aggFuncRefs.get(i).getValue(), context);
+                    .createMergeAggregation(aggProducedVars.get(i), aggFuncExpr, context);
             if (mergeExpr == null) {
-                throw new AlgebricksException("The aggregation function " + aggFuncRefs.get(i).getValue()
-                        + " does not have a registered intermediate aggregation function.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, aggFuncExpr.getSourceLocation(),
+                        "The aggregation function " + aggFuncExpr
+                                + " does not have a registered intermediate aggregation function.");
             }
             mergeExpressionRefs.add(new MutableObject<ILogicalExpression>(mergeExpr));
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
index 48a744d..52858e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.optimizer.rules;
 
 import org.apache.asterix.common.config.GlobalConfig;
+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.functions.BuiltinFunctions;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
@@ -110,7 +112,7 @@
                 if (reqType == null || !reqType.isOpen()) {
                     int n = expr.getArguments().size();
                     if (n % 2 > 0) {
-                        throw new AlgebricksException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
                                 "Record constructor expected to have an even number of arguments: " + expr);
                     }
                     for (int i = 0; i < n / 2; i++) {
@@ -166,7 +168,7 @@
                 throws AlgebricksException {
             Object varType = env.getVarType(expr.getVariableReference());
             if (varType == null) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
                         "Could not infer type for variable '" + expr.getVariableReference() + "'.");
             }
             return new ClosedDataInfo(false, TypeHelper.isClosed((IAType) varType), expr);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
index 90491d3..cc50dce4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
@@ -104,6 +104,7 @@
 
         //create ExtensionOperator and put the commitOperator in it.
         DelegateOperator extensionOperator = new DelegateOperator(commitOperator);
+        extensionOperator.setSourceLocation(eOp.getSourceLocation());
         extensionOperator.setPhysicalOperator(commitPOperator);
 
         //update plan link
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
index 0c3de91..d4061e0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
@@ -50,6 +50,7 @@
 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.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Looks for a select operator, containing a condition:
@@ -181,10 +182,12 @@
 
         // Only non-null if we found that varRefExpr refers to an optimizable similarity function call.
         if (simCheckFuncExpr != null) {
+            SourceLocation sourceLoc = simCheckFuncExpr.getSourceLocation();
             // Create a new assign under matchingAssign which assigns the result of our similarity-check function to a variable.
             LogicalVariable newVar = context.newVar();
             AssignOperator newAssign =
                     new AssignOperator(newVar, new MutableObject<ILogicalExpression>(simCheckFuncExpr));
+            newAssign.setSourceLocation(sourceLoc);
             // Hook up inputs.
             newAssign.getInputs()
                     .add(new MutableObject<ILogicalOperator>(matchingAssign.getInputs().get(0).getValue()));
@@ -193,12 +196,15 @@
             // Replace select condition with a get-item on newVarFromExpression.
             List<Mutable<ILogicalExpression>> selectGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVarFromExpression.
-            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            VariableReferenceExpression newVarRef1 = new VariableReferenceExpression(newVar);
+            newVarRef1.setSourceLocation(sourceLoc);
+            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(newVarRef1));
             // Second arg is the item index to be accessed, here 0.
             selectGetItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression selectGetItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression selectGetItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), selectGetItemArgs);
+            selectGetItemExpr.setSourceLocation(sourceLoc);
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(selectGetItemExpr);
 
@@ -206,12 +212,15 @@
             // newVarFromExpression.
             List<Mutable<ILogicalExpression>> assignGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVarFromExpression.
-            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            VariableReferenceExpression newVarRef2 = new VariableReferenceExpression(newVar);
+            newVarRef2.setSourceLocation(sourceLoc);
+            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(newVarRef2));
             // Second arg is the item index to be accessed, here 1.
             assignGetItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
-            ILogicalExpression assignGetItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression assignGetItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), assignGetItemArgs);
+            assignGetItemExpr.setSourceLocation(sourceLoc);
             // Replace the original assign expr with the get-item expr.
             simFuncExprRef.setValue(assignGetItemExpr);
 
@@ -238,8 +247,9 @@
             // Second arg is the item index to be accessed.
             getItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression getItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression getItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), getItemArgs);
+            getItemExpr.setSourceLocation(simCheckFuncExpr.getSourceLocation());
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(getItemExpr);
             return true;
@@ -282,6 +292,7 @@
                     new ConstantExpression(new AsterixConstantValue(jaccThresh))));
             simCheckFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.SIMILARITY_JACCARD_CHECK), similarityArgs);
+            simCheckFuncExpr.setSourceLocation(funcExpr.getSourceLocation());
         }
 
         // Look for edit-distance function call, and LE or LT.
@@ -310,6 +321,7 @@
                     new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(edThresh))));
             simCheckFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.EDIT_DISTANCE_CHECK), similarityArgs);
+            simCheckFuncExpr.setSourceLocation(funcExpr.getSourceLocation());
         }
         // Preserve all annotations.
         if (simCheckFuncExpr != null) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index b1f646a..8372851 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -20,6 +20,8 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -102,7 +104,7 @@
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 if (!expr.isFunctional()) {
                     AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
                             "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
                 }
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 4f9b4df..c8bec85 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -96,7 +96,7 @@
                     BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
                     BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
                     BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
-                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_MOD);
+                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
 
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
@@ -625,7 +625,9 @@
         optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
         List<String> fieldName = null;
         if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
-            optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+            varRef.setSourceLocation(unnestOp.getSourceLocation());
+            optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
         } else {
             fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
                     funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
@@ -731,7 +733,9 @@
             optFuncExpr.setFieldName(funcVarIndex, fieldName);
             optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
             optFuncExpr.setSourceVar(funcVarIndex, var);
-            optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+            varRef.setSourceLocation(subTree.getDataSourceRef().getValue().getSourceLocation());
+            optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
             setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
             if (subTree.hasDataSourceScan()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index e541a79..cba4411 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -107,8 +107,8 @@
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(varList.size()))));
         funcArgs.add(numKeysRef);
         for (LogicalVariable keyVar : varList) {
-            Mutable<ILogicalExpression> keyVarRef = new MutableObject<>(new VariableReferenceExpression(keyVar));
-            funcArgs.add(keyVarRef);
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            funcArgs.add(new MutableObject<>(keyVarRef));
         }
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index eafbdaf..6368058 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -95,6 +95,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 
 /**
@@ -274,7 +275,8 @@
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED,
-                        BuiltinFunctions.FULLTEXT_CONTAINS.getName(), objectFromExpr.getType().getTypeTag());
+                        constExpression.getSourceLocation(), BuiltinFunctions.FULLTEXT_CONTAINS.getName(),
+                        objectFromExpr.getType().getTypeTag());
         }
     }
 
@@ -421,7 +423,8 @@
             case CONDITIONAL_SPLIT_VAR:
                 // Sanity check - the given unnest map should generate this variable.
                 if (!abstractUnnestMapOp.getGenerateCallBackProceedResultVar()) {
-                    throw CompilationException.create(ErrorCode.CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE);
+                    throw CompilationException.create(ErrorCode.CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE,
+                            unnestMapOp.getSourceLocation());
                 }
                 // Fetches conditional splitter - the last position
                 start = numSecondaryKeys + numPrimaryKeys;
@@ -465,6 +468,7 @@
     public static Triple<ILogicalExpression, ILogicalExpression, Boolean> createSearchKeyExpr(Index index,
             IOptimizableFuncExpr optFuncExpr, IAType indexedFieldType, OptimizableOperatorSubTree probeSubTree)
             throws AlgebricksException {
+        SourceLocation sourceLoc = optFuncExpr.getFuncExpr().getSourceLocation();
         if (probeSubTree == null) {
             // We are optimizing a selection query. Search key is a constant.
             // Type Checking and type promotion is done here
@@ -473,7 +477,9 @@
                 //We are looking at a selection case, but using two variables
                 //This means that the second variable comes from a nonPure function call
                 //TODO: Right now we miss on type promotion for nonpure functions
-                return new Triple<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), null, false);
+                VariableReferenceExpression varRef = new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
+                varRef.setSourceLocation(sourceLoc);
+                return new Triple<>(varRef, null, false);
             }
 
             ILogicalExpression constantAtRuntimeExpression = optFuncExpr.getConstantExpr(0);
@@ -539,25 +545,25 @@
                         case LT:
                         case GE:
                             // round-up
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.CEIL, sourceLoc);
                             break;
                         case LE:
                         case GT:
                             // round-down
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.FLOOR, sourceLoc);
                             break;
                         case EQ:
                             // equality case - both CEIL and FLOOR need to be applied.
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
-                            replacedConstantValueForEQCase =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.FLOOR, sourceLoc);
+                            replacedConstantValueForEQCase = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.CEIL, sourceLoc);
                             break;
                         default:
                             // NEQ should not be a case.
@@ -566,7 +572,7 @@
                 } else {
                     // Type conversion only case: (e.g., INT -> BIGINT)
                     replacedConstantValue = getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.NONE);
+                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.NONE, sourceLoc);
                 }
             }
             // No type-casting at all
@@ -586,7 +592,8 @@
             OptimizableOperatorSubTree opSubTree0 = optFuncExpr.getOperatorSubTree(0);
             int probeVarIndex = opSubTree0 == null || opSubTree0 == probeSubTree ? 0 : 1;
             LogicalVariable probeVar = optFuncExpr.getLogicalVar(probeVarIndex);
-            ILogicalExpression probeExpr = new VariableReferenceExpression(probeVar);
+            VariableReferenceExpression probeExpr = new VariableReferenceExpression(probeVar);
+            probeExpr.setSourceLocation(sourceLoc);
 
             ATypeTag indexedFieldTypeTag = TypeComputeUtils.getActualType(indexedFieldType).getTypeTag();
             if (ATypeHierarchy.getTypeDomain(indexedFieldTypeTag) == ATypeHierarchy.Domain.NUMERIC) {
@@ -595,6 +602,7 @@
                 if (probeTypeTypeTag != indexedFieldTypeTag) {
                     ScalarFunctionCallExpression castFunc = new ScalarFunctionCallExpression(
                             FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE_LAX));
+                    castFunc.setSourceLocation(sourceLoc);
                     castFunc.getArguments().add(new MutableObject<>(probeExpr));
                     TypeCastUtils.setRequiredAndInputTypes(castFunc, indexedFieldType, probeType);
                     boolean realTypeConvertedToIntegerType =
@@ -607,14 +615,14 @@
     }
 
     private static AsterixConstantValue getReplacedConstantValue(IAObject sourceObject, ATypeTag sourceTypeTag,
-            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction)
-            throws CompilationException {
+            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction,
+            SourceLocation sourceLoc) throws CompilationException {
         try {
             return ATypeHierarchy.getAsterixConstantValueFromNumericTypeObject(sourceObject, targetTypeTag,
                     strictDemote, mathFunction);
         } catch (HyracksDataException e) {
-            throw new CompilationException(ErrorCode.ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION, e, sourceTypeTag,
-                    targetTypeTag);
+            throw new CompilationException(ErrorCode.ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION, e, sourceLoc,
+                    sourceTypeTag, targetTypeTag);
         }
     }
 
@@ -749,6 +757,7 @@
                 // Non-index only plan case
                 indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
                 SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+                topSelectOp.setSourceLocation(finalIndexSearchOp.getSourceLocation());
                 topSelectOp.getInputs().add(indexSubTree.getRootRef());
                 topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
                 context.computeAndSetTypeEnvironmentForOperator(topSelectOp);
@@ -768,6 +777,7 @@
             ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
             IOptimizationContext context, boolean retainInput, boolean retainNull,
             boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
         jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
@@ -784,6 +794,7 @@
         IFunctionInfo secondaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         UnnestingFunctionCallExpression secondaryIndexSearchFunc =
                 new UnnestingFunctionCallExpression(secondaryIndexSearch, secondaryIndexFuncArgs);
+        secondaryIndexSearchFunc.setSourceLocation(sourceLoc);
         secondaryIndexSearchFunc.setReturnsUniqueValues(true);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has all
         // necessary arguments to determine which index to use, which variables contain the index-search keys,
@@ -796,6 +807,7 @@
                 LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
                         secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
                         secondaryIndexOutputTypes, true);
+                secondaryIndexLeftOuterUnnestOp.setSourceLocation(sourceLoc);
                 secondaryIndexLeftOuterUnnestOp
                         .setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
                 secondaryIndexLeftOuterUnnestOp.getInputs().add(new MutableObject<>(inputOp));
@@ -804,13 +816,15 @@
                 return secondaryIndexLeftOuterUnnestOp;
             } else {
                 // Left-outer-join without retainInput doesn't make sense.
-                throw new AlgebricksException("Left-outer-join should propagate all inputs from the outer branch.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Left-outer-join should propagate all inputs from the outer branch.");
             }
         } else {
             // If this is not a left-outer-join case, then we use UNNEST-MAP operator.
             UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes,
                     retainInput);
+            secondaryIndexUnnestOp.setSourceLocation(sourceLoc);
             secondaryIndexUnnestOp.setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
             secondaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
             context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
@@ -824,12 +838,16 @@
             boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
             List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // Optionally add a sort on the primary-index keys before searching the primary index.
         OrderOperator order = null;
         if (sortPrimaryKeys) {
             order = new OrderOperator();
+            order.setSourceLocation(sourceLoc);
             for (LogicalVariable pkVar : primaryKeyVars) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<>(new VariableReferenceExpression(pkVar));
+                VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+                pkVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
                 order.getOrderExpressions().add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
             }
             // The secondary-index search feeds into the sort.
@@ -837,9 +855,11 @@
             order.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(order);
         }
+
         // Creates the primary-index search unnest-map operator.
-        AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
-                retainMissing, requiresBroadcast, primaryKeyVars, primaryIndexUnnestVars, primaryIndexOutputTypes);
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp =
+                createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
+                        primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
         if (sortPrimaryKeys) {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
         } else {
@@ -859,6 +879,7 @@
             LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
             List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
         // From now on, we deal with the index-only plan.
         // Initializes the information required for the index-only plan optimization.
@@ -930,19 +951,24 @@
             switch (spatialType.getTypeTag()) {
                 case POINT:
                     // Reconstructs a POINT value.
-                    AbstractFunctionCallExpression createPointExpr = createPointExpression(skVarsFromSIdxUnnestMap);
+                    AbstractFunctionCallExpression createPointExpr =
+                            createPointExpression(skVarsFromSIdxUnnestMap, sourceLoc);
                     restoredSKVarFromRTree.add(context.newVar());
                     restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createPointExpr));
                     skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    skVarAssignOpInRightPath.setSourceLocation(sourceLoc);
                     break;
                 case RECTANGLE:
                     // Reconstructs a RECTANGLE value.
-                    AbstractFunctionCallExpression expr1 = createPointExpression(skVarsFromSIdxUnnestMap.subList(0, 2));
-                    AbstractFunctionCallExpression expr2 = createPointExpression(skVarsFromSIdxUnnestMap.subList(2, 4));
+                    AbstractFunctionCallExpression expr1 =
+                            createPointExpression(skVarsFromSIdxUnnestMap.subList(0, 2), sourceLoc);
+                    AbstractFunctionCallExpression expr2 =
+                            createPointExpression(skVarsFromSIdxUnnestMap.subList(2, 4), sourceLoc);
                     AbstractFunctionCallExpression createRectangleExpr = createRectangleExpression(expr1, expr2);
                     restoredSKVarFromRTree.add(context.newVar());
                     restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createRectangleExpr));
                     skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    skVarAssignOpInRightPath.setSourceLocation(sourceLoc);
                     break;
                 default:
                     break;
@@ -1022,6 +1048,7 @@
                 // If this assign operator is not used in the SELECT or JOIN operator,
                 // we will add this operator after creating UNION operator in the last part of this method.
                 constAssignOp = new AssignOperator(constAssignVars, constAssignExprs);
+                constAssignOp.setSourceLocation(sourceLoc);
                 if (constantAssignVarUsedInTopOp) {
                     // Places this assign after the secondary index-search op.
                     constAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
@@ -1052,6 +1079,7 @@
         // Adds a SPLIT operator after the given secondary index-search unnest-map operator.
         splitOp = new SplitOperator(2,
                 new MutableObject<ILogicalExpression>(new VariableReferenceExpression(condSplitVars.get(0))));
+        splitOp.setSourceLocation(sourceLoc);
         splitOp.getInputs().add(new MutableObject<ILogicalOperator>(currentOp));
         splitOp.setExecutionMode(ExecutionMode.PARTITIONED);
         context.computeAndSetTypeEnvironmentForOperator(splitOp);
@@ -1075,9 +1103,12 @@
             LogicalVariable newVar = context.newVar();
             liveVarAfterSplitToLeftPathMap.put(v, newVar);
             assignVars.add(newVar);
-            assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            assignExprs.add(new MutableObject<ILogicalExpression>(vRef));
         }
         AssignOperator origVarsToLeftPathVarsAssignOp = new AssignOperator(assignVars, assignExprs);
+        origVarsToLeftPathVarsAssignOp.setSourceLocation(sourceLoc);
         origVarsToLeftPathVarsAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(splitOp));
         context.computeAndSetTypeEnvironmentForOperator(origVarsToLeftPathVarsAssignOp);
         origVarsToLeftPathVarsAssignOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -1199,7 +1230,8 @@
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
                 retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
-                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes);
+                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc);
+        primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
         context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
         primaryIndexUnnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -1217,6 +1249,7 @@
         LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
         newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
                 retainMissing, newMissingPlaceHolderVar);
+        newSelectOpInLeftPath.setSourceLocation(conditionRefExpr.getSourceLocation());
         VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
 
         // If there are ASSIGN operators before the SELECT or JOIN operator,
@@ -1276,6 +1309,7 @@
             ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
             newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
                     retainMissing, newMissingPlaceHolderVar);
+            newSelectOpInRightPath.setSourceLocation(conditionRefExpr2.getSourceLocation());
             newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
             VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
             VariableUtilities.substituteVariables(newSelectOpInRightPath, origSKFieldVarToNewSKFieldVarMap, context);
@@ -1294,6 +1328,7 @@
 
         // UNIONALL operator that combines both paths.
         unionAllOp = new UnionAllOperator(unionVarMap);
+        unionAllOp.setSourceLocation(sourceLoc);
         unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(newSelectOpInLeftPath));
         unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
 
@@ -1322,8 +1357,8 @@
 
     private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
             boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
-            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
-            throws AlgebricksException {
+            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes,
+            SourceLocation sourceLoc) throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -1339,6 +1374,7 @@
         IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         AbstractFunctionCallExpression primaryIndexSearchFunc =
                 new ScalarFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
+        primaryIndexSearchFunc.setSourceLocation(sourceLoc);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has
         // all necessary arguments to determine which index to use, which variables contain the index-search keys,
         // what is the original dataset, etc.
@@ -1348,14 +1384,17 @@
                 primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
                         new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                         retainInput);
+                primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
             } else {
                 // Left-outer-join without retainNull and retainInput doesn't make sense.
-                throw new AlgebricksException("Left-outer-join should propagate all inputs from the outer branch.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Left-outer-join should propagate all inputs from the outer branch.");
             }
         } else {
             primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                     retainInput);
+            primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         }
         return primaryIndexUnnestMapOp;
     }
@@ -1416,12 +1455,18 @@
         }
     }
 
-    private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars) {
+    private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars,
+            SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         AbstractFunctionCallExpression createPointExpr1 =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_POINT));
-        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(0))));
-        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(1))));
+        createPointExpr1.setSourceLocation(sourceLoc);
+        VariableReferenceExpression pointVarRef0 = new VariableReferenceExpression(pointVars.get(0));
+        pointVarRef0.setSourceLocation(sourceLoc);
+        expressions.add(new MutableObject<ILogicalExpression>(pointVarRef0));
+        VariableReferenceExpression pointVarRef1 = new VariableReferenceExpression(pointVars.get(1));
+        pointVarRef1.setSourceLocation(sourceLoc);
+        expressions.add(new MutableObject<ILogicalExpression>(pointVarRef1));
         createPointExpr1.getArguments().addAll(expressions);
         return createPointExpr1;
     }
@@ -1431,6 +1476,7 @@
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         AbstractFunctionCallExpression createRectangleExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_RECTANGLE));
+        createRectangleExpr.setSourceLocation(createPointExpr1.getSourceLocation());
         expressions.add(new MutableObject<ILogicalExpression>(createPointExpr1));
         expressions.add(new MutableObject<ILogicalExpression>(createPointExpr2));
         createRectangleExpr.getArguments().addAll(expressions);
@@ -1475,7 +1521,8 @@
         }
 
         if (!foundSelectNonMissing) {
-            throw CompilationException.create(ErrorCode.CANNOT_FIND_NON_MISSING_SELECT_OPERATOR);
+            throw CompilationException.create(ErrorCode.CANNOT_FIND_NON_MISSING_SELECT_OPERATOR,
+                    lojGroupbyOp.getSourceLocation());
         }
         return isMissingFuncExpr;
     }
@@ -1486,8 +1533,9 @@
         //reset the missing placeholder variable in groupby operator
         ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInGroupBy();
         isMissingFuncExpr.getArguments().clear();
-        isMissingFuncExpr.getArguments().add(
-                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newMissingPlaceholderVaraible)));
+        VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible);
+        newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation());
+        isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef));
 
         //recompute type environment.
         OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJGroupbyOpRef(), context);
@@ -1526,12 +1574,16 @@
             Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
             IOptimizationContext context, Index secondaryIndex, boolean retainInput, boolean retainNull)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset, recordType,
                 metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
         // add a sort on the RID fields before fetching external data.
         OrderOperator order = new OrderOperator();
+        order.setSourceLocation(sourceLoc);
         for (LogicalVariable pkVar : primaryKeyVars) {
-            Mutable<ILogicalExpression> vRef = new MutableObject<>(new VariableReferenceExpression(pkVar));
+            VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+            pkVarRef.setSourceLocation(sourceLoc);
+            Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
             order.getOrderExpressions().add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
         }
         // The secondary-index search feeds into the sort.
@@ -1556,8 +1608,10 @@
         IFunctionInfo externalLookup = FunctionUtil.getFunctionInfo(BuiltinFunctions.EXTERNAL_LOOKUP);
         AbstractFunctionCallExpression externalLookupFunc =
                 new ScalarFunctionCallExpression(externalLookup, externalLookupArgs);
+        externalLookupFunc.setSourceLocation(sourceLoc);
         UnnestMapOperator unnestOp = new UnnestMapOperator(externalUnnestVars,
                 new MutableObject<ILogicalExpression>(externalLookupFunc), outputTypes, retainInput);
+        unnestOp.setSourceLocation(sourceLoc);
         // Fed by the order operator or the secondaryIndexUnnestOp.
         unnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 60d4d3d..099f769 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -68,6 +68,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -353,7 +354,8 @@
                 keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
             }
             if (keyPos < 0) {
-                throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR);
+                throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR,
+                        optFuncExpr.getFuncExpr().getSourceLocation());
             }
             // returnedSearchKeyExpr contains a pair of search expression.
             // The second expression will not be null only if we are creating an EQ search predicate
@@ -368,7 +370,9 @@
             if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 constantAtRuntimeExpressions[keyPos] = searchKeyExpr;
                 constAtRuntimeExprVars[keyPos] = context.newVar();
-                searchKeyExpr = new VariableReferenceExpression(constAtRuntimeExprVars[keyPos]);
+                VariableReferenceExpression varRef = new VariableReferenceExpression(constAtRuntimeExprVars[keyPos]);
+                varRef.setSourceLocation(searchKeyExpr.getSourceLocation());
+                searchKeyExpr = varRef;
             }
 
             LimitType limit = getLimitType(optFuncExpr, probeSubTree);
@@ -576,6 +580,7 @@
         if (!assignKeyVarList.isEmpty()) {
             // Assign operator that sets the constant secondary-index search-key fields if necessary.
             AssignOperator assignSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
+            assignSearchKeys.setSourceLocation(dataSourceOp.getSourceLocation());
             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).
@@ -694,26 +699,33 @@
                 IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
                 UnnestingFunctionCallExpression primaryIndexSearchFunc =
                         new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
+                primaryIndexSearchFunc.setSourceLocation(dataSourceOp.getSourceLocation());
                 primaryIndexSearchFunc.setReturnsUniqueValues(true);
+                AbstractUnnestMapOperator unnestMapOp;
                 if (!leftOuterUnnestMapRequired) {
-                    indexSearchOp = new UnnestMapOperator(scanVariables,
+                    unnestMapOp = new UnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             true);
                 }
+                unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+                indexSearchOp = unnestMapOp;
             } else {
+                AbstractUnnestMapOperator unnestMapOp;
                 if (!leftOuterUnnestMapRequired) {
-                    indexSearchOp = new UnnestMapOperator(scanVariables,
+                    unnestMapOp = new UnnestMapOperator(scanVariables,
                             ((UnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             ((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
                             primaryIndexOutputTypes, true);
                 }
+                unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+                indexSearchOp = unnestMapOp;
             }
 
             indexSearchOp.getInputs().add(new MutableObject<>(inputOp));
@@ -775,7 +787,7 @@
         // The original select cond must be an AND. Check it just to be sure.
         if (funcExpr.getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND) {
             throw new CompilationException(ErrorCode.COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX,
-                    funcExpr.toString());
+                    funcExpr.getSourceLocation(), funcExpr.toString());
         }
         // Clean the conjuncts.
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
@@ -902,7 +914,9 @@
     private ILogicalExpression createSelectCondition(List<Mutable<ILogicalExpression>> predList) {
         if (predList.size() > 1) {
             IFunctionInfo finfo = FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND);
-            return new ScalarFunctionCallExpression(finfo, predList);
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(finfo, predList);
+            andExpr.setSourceLocation(predList.get(0).getValue().getSourceLocation());
+            return andExpr;
         }
         return predList.get(0).getValue();
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 7938f49..7739833 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -66,6 +66,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
 
@@ -134,7 +135,8 @@
     }
 
     private AssignOperator createAssignOperator(List<IOptimizableFuncExpr> optFuncExprs,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context) {
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context,
+            SourceLocation sourceLoc) {
         List<LogicalVariable> assignKeyVarList = new ArrayList<>();
         List<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<>();
 
@@ -154,7 +156,9 @@
                 maxFilterVars.add(var);
             }
         }
-        return new AssignOperator(assignKeyVarList, assignKeyExprList);
+        AssignOperator assignOp = new AssignOperator(assignKeyVarList, assignKeyExprList);
+        assignOp.setSourceLocation(sourceLoc);
+        return assignOp;
     }
 
     private void assignFilterFromQuery(List<IOptimizableFuncExpr> optFuncExprs, AbstractLogicalOperator op,
@@ -175,15 +179,17 @@
                     List<LogicalVariable> minFilterVars = new ArrayList<>();
                     List<LogicalVariable> maxFilterVars = new ArrayList<>();
 
-                    AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
+                    AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context,
+                            dataSourceScanOp.getSourceLocation());
 
                     dataSourceScanOp.setMinFilterVars(minFilterVars);
                     dataSourceScanOp.setMaxFilterVars(maxFilterVars);
 
                     List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
                     for (LogicalVariable var : assignOp.getVariables()) {
-                        additionalFilteringExpressions
-                                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                        varRef.setSourceLocation(assignOp.getSourceLocation());
+                        additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                     }
 
                     dataSourceScanOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
@@ -206,16 +212,17 @@
                         List<LogicalVariable> minFilterVars = new ArrayList<>();
                         List<LogicalVariable> maxFilterVars = new ArrayList<>();
 
-                        AssignOperator assignOp =
-                                createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
+                        AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars,
+                                context, unnestMapOp.getSourceLocation());
 
                         unnestMapOp.setMinFilterVars(minFilterVars);
                         unnestMapOp.setMaxFilterVars(maxFilterVars);
 
                         List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
                         for (LogicalVariable var : assignOp.getVariables()) {
-                            additionalFilteringExpressions
-                                    .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                            varRef.setSourceLocation(assignOp.getSourceLocation());
+                            additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                         }
                         unnestMapOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                         assignOp.getInputs().add(new MutableObject<>(unnestMapOp.getInputs().get(0).getValue()));
@@ -320,6 +327,7 @@
         }
 
         IntersectOperator intersectWithFilter = new IntersectOperator(outputVars, compareVars, filterVars);
+        intersectWithFilter.setSourceLocation(intersect.getSourceLocation());
         intersectWithFilter.getInputs().addAll(intersect.getInputs());
         return intersectWithFilter;
     }
@@ -343,9 +351,12 @@
         primaryOp.setMinFilterVars(Collections.singletonList(minFilterVar));
         primaryOp.setMaxFilterVars(Collections.singletonList(maxFilterVar));
 
+        VariableReferenceExpression minFilterVarRef = new VariableReferenceExpression(minFilterVar);
+        minFilterVarRef.setSourceLocation(primaryOp.getSourceLocation());
+        VariableReferenceExpression maxFilterVarRef = new VariableReferenceExpression(maxFilterVar);
+        maxFilterVarRef.setSourceLocation(primaryOp.getSourceLocation());
         List<Mutable<ILogicalExpression>> indexFilterExpression =
-                Arrays.asList(new MutableObject<>(new VariableReferenceExpression(minFilterVar)),
-                        new MutableObject<>(new VariableReferenceExpression(maxFilterVar)));
+                Arrays.asList(new MutableObject<>(minFilterVarRef), new MutableObject<>(maxFilterVarRef));
 
         primaryOp.setAdditionalFilteringExpressions(indexFilterExpression);
         context.computeAndSetTypeEnvironmentForOperator(primaryOp);
@@ -366,7 +377,8 @@
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                     FunctionIdentifier fid = f.getFunctionIdentifier();
                     if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fid.getName());
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(),
+                                fid.getName());
                     }
                     AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                     jobGenParams.readFromFuncArgs(f.getArguments());
@@ -414,7 +426,8 @@
                         dataverseName = jobGenParams.dataverseName;
                         datasetName = jobGenParams.datasetName;
                     } else {
-                        throw new AlgebricksException("Unexpected function for Unnest Map: " + fid);
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, f.getSourceLocation(),
+                                "Unexpected function for Unnest Map: " + fid);
                     }
                     return ((MetadataProvider) context.getMetadataProvider()).findDataset(dataverseName, datasetName);
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 27e5645..d2b456f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -262,10 +262,12 @@
         List<List<LogicalVariable>> inputVars = new ArrayList<>(subRoots.size());
         for (int i = 0; i < subRoots.size(); i++) {
             if (lop.getOperatorTag() != subRoots.get(i).getOperatorTag()) {
-                throw new AlgebricksException("The data source root should have the same operator type.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, lop.getSourceLocation(),
+                        "The data source root should have the same operator type.");
             }
             if (lop.getInputs().size() != 1) {
-                throw new AlgebricksException("The primary search has multiple inputs.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, lop.getSourceLocation(),
+                        "The primary search has multiple inputs.");
             }
 
             ILogicalOperator curRoot = subRoots.get(i);
@@ -274,7 +276,8 @@
             for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> orderExpression : order
                     .getOrderExpressions()) {
                 if (orderExpression.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                            orderExpression.second.getValue().getSourceLocation(),
                             "The order by expression should be variables, but they aren't variables.");
                 }
                 VariableReferenceExpression orderedVar =
@@ -286,6 +289,7 @@
 
         List<LogicalVariable> outputVar = inputVars.get(0);
         IntersectOperator intersect = new IntersectOperator(outputVar, inputVars);
+        intersect.setSourceLocation(lop.getSourceLocation());
         for (ILogicalOperator secondarySearch : subRoots) {
             intersect.getInputs().add(secondarySearch.getInputs().get(0));
         }
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.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 @@
             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 @@
         // 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 @@
                         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 @@
         // 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 @@
     }
 
     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 @@
             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 @@
             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 @@
             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 @@
                         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 @@
         // 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 @@
                 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 @@
                         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 @@
                         }
                     }
                     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);
         }
     }
 
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 @@
             // 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 @@
                 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 @@
                 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 @@
                     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;
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 f4316037..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 @@
             // 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 @@
         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);
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..4d646f3 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.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.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 @@
         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 @@
         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 @@
         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 @@
             // 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 @@
         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 @@
         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 @@
         // 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 @@
         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 @@
                 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 @@
         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;
@@ -529,8 +562,8 @@
         // Update the variable mappings
         List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varTriples = op.getVariableMappings();
         for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple : varTriples) {
-            updateInputToOutputVarMapping(triple.third, triple.first, false);
-            updateInputToOutputVarMapping(triple.second, triple.first, false);
+            updateInputToOutputVarMapping(triple.first, triple.third, false);
+            updateInputToOutputVarMapping(triple.second, triple.third, false);
         }
         return op;
     }
@@ -542,7 +575,8 @@
         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 @@
     @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 @@
         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 @@
             // 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 @@
         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));
 
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 @@
         // 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 @@
         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 @@
      */
     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);
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.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 @@
                 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 @@
 
         // 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 @@
         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 @@
         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 @@
             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 @@
         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 @@
     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 @@
         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 @@
             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 @@
             // 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 = nullCheckExprRefs.size() > 1
-                    ? new MutableObject<>(new ScalarFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), nullCheckExprRefs))
-                    : nullCheckExprRefs.get(0);
+            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);
+            }
             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);
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 @@
     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 @@
         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;
     }
 
 }
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.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 @@
                 }
             }
             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;
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 @@
                 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);
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 @@
             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);
     }
 
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.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 @@
 public class CompiledStatements {
 
     public interface ICompiledStatement {
-
         Statement.Kind getKind();
+
+        SourceLocation getSourceLocation();
     }
 
-    public static class CompiledDatasetDropStatement implements ICompiledStatement {
+    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 extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
 
@@ -64,7 +78,7 @@
     }
 
     // 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 static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+    public static class CompiledNodeGroupDropStatement extends AbstractCompiledStatement {
         private final String nodeGroupName;
 
         public CompiledNodeGroupDropStatement(String nodeGroupName) {
@@ -104,7 +118,7 @@
         }
     }
 
-    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 static class CompiledDataverseDropStatement implements ICompiledStatement {
+    public static class CompiledDataverseDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final boolean ifExists;
 
@@ -156,7 +170,7 @@
         }
     }
 
-    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 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 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 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 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 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 static class CompiledCompactStatement implements ICompiledStatement {
+    public static class CompiledCompactStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
new file mode 100644
index 0000000..d77164c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.translator;
+
+import java.io.Serializable;
+
+public class ExecutionPlans implements Serializable {
+
+    private String expressionTree;
+    private String rewrittenExpressionTree;
+    private String logicalPlan;
+    private String optimizedLogicalPlan;
+    private String job;
+
+    public String getExpressionTree() {
+        return expressionTree;
+    }
+
+    public void setExpressionTree(String expressionTree) {
+        this.expressionTree = expressionTree;
+    }
+
+    public String getRewrittenExpressionTree() {
+        return rewrittenExpressionTree;
+    }
+
+    public void setRewrittenExpressionTree(String rewrittenExpressionTree) {
+        this.rewrittenExpressionTree = rewrittenExpressionTree;
+    }
+
+    public String getLogicalPlan() {
+        return logicalPlan;
+    }
+
+    public void setLogicalPlan(String logicalPlan) {
+        this.logicalPlan = logicalPlan;
+    }
+
+    public String getOptimizedLogicalPlan() {
+        return optimizedLogicalPlan;
+    }
+
+    public void setOptimizedLogicalPlan(String optimizedLogicalPlan) {
+        this.optimizedLogicalPlan = optimizedLogicalPlan;
+    }
+
+    public String getJob() {
+        return job;
+    }
+
+    public void setJob(String job) {
+        this.job = job;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java
new file mode 100644
index 0000000..88e8255
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.translator;
+
+import java.io.PrintWriter;
+
+public class ExecutionPlansHtmlPrintUtil {
+
+    private static final String LOGICAL_PLAN_LBL = "Logical plan";
+    private static final String EXPRESSION_TREE_LBL = "Expression tree";
+    private static final String REWRITTEN_EXPRESSION_TREE_LBL = "Rewritten expression tree";
+    private static final String OPTIMIZED_LOGICAL_PLAN_LBL = "Optimized logical plan";
+    private static final String JOB_LBL = "Job";
+
+    private ExecutionPlansHtmlPrintUtil() {
+    }
+
+    public static void print(PrintWriter output, ExecutionPlans plans) {
+        printNonNull(output, EXPRESSION_TREE_LBL, plans.getExpressionTree());
+        printNonNull(output, REWRITTEN_EXPRESSION_TREE_LBL, plans.getRewrittenExpressionTree());
+        printNonNull(output, LOGICAL_PLAN_LBL, plans.getLogicalPlan());
+        printNonNull(output, OPTIMIZED_LOGICAL_PLAN_LBL, plans.getOptimizedLogicalPlan());
+        printNonNull(output, JOB_LBL, plans.getJob());
+    }
+
+    private static void printNonNull(PrintWriter output, String lbl, String value) {
+        if (value != null) {
+            printFieldPrefix(output, lbl);
+            output.print(value);
+            printFieldPostfix(output);
+        }
+    }
+
+    private static void printFieldPrefix(PrintWriter output, String lbl) {
+        output.println();
+        output.println("<h4>" + lbl + ":</h4>");
+        switch (lbl) {
+            case LOGICAL_PLAN_LBL:
+                output.println("<pre class=query-plan>");
+                break;
+            case OPTIMIZED_LOGICAL_PLAN_LBL:
+                output.println("<pre class=query-optimized-plan>");
+                break;
+            default:
+                output.println("<pre>");
+                break;
+        }
+    }
+
+    private static void printFieldPostfix(PrintWriter output) {
+        output.println("</pre>");
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
new file mode 100644
index 0000000..5c47ca2
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.translator;
+
+import static org.apache.asterix.translator.SessionConfig.PlanFormat.STRING;
+
+import org.apache.hyracks.util.JSONUtil;
+
+public class ExecutionPlansJsonPrintUtil {
+
+    private static final String LOGICAL_PLAN_LBL = "logicalPlan";
+    private static final String EXPRESSION_TREE_LBL = "expressionTree";
+    private static final String REWRITTEN_EXPRESSION_TREE_LBL = "rewrittenExpressionTree";
+    private static final String OPTIMIZED_LOGICAL_PLAN_LBL = "optimizedLogicalPlan";
+    private static final String JOB_LBL = "job";
+
+    private ExecutionPlansJsonPrintUtil() {
+    }
+
+    public static String asJson(ExecutionPlans plans, SessionConfig.PlanFormat format) {
+        final StringBuilder output = new StringBuilder();
+        appendOutputPrefix(output);
+        // TODO only string is currently supported for expression trees
+        appendNonNull(output, EXPRESSION_TREE_LBL, plans.getExpressionTree(), STRING);
+        appendNonNull(output, REWRITTEN_EXPRESSION_TREE_LBL, plans.getRewrittenExpressionTree(), STRING);
+        appendNonNull(output, LOGICAL_PLAN_LBL, plans.getLogicalPlan(), format);
+        appendNonNull(output, OPTIMIZED_LOGICAL_PLAN_LBL, plans.getOptimizedLogicalPlan(), format);
+        appendNonNull(output, JOB_LBL, plans.getJob(), format);
+        appendOutputPostfix(output);
+        return output.toString();
+    }
+
+    private static void appendNonNull(StringBuilder builder, String lbl, String value,
+            SessionConfig.PlanFormat format) {
+        if (value != null) {
+            printFieldPrefix(builder, lbl);
+            switch (format) {
+                case JSON:
+                    builder.append(value);
+                    break;
+                case STRING:
+                    JSONUtil.quoteAndEscape(builder, value);
+                    break;
+                default:
+                    throw new IllegalStateException("Unrecognized plan format: " + format);
+            }
+            printFieldPostfix(builder);
+        }
+    }
+
+    private static void appendOutputPrefix(StringBuilder builder) {
+        builder.append("{");
+    }
+
+    private static void printFieldPrefix(StringBuilder builder, String lbl) {
+        builder.append("\"" + lbl + "\": ");
+    }
+
+    private static void printFieldPostfix(StringBuilder builder) {
+        builder.append(",");
+    }
+
+    private static void appendOutputPostfix(StringBuilder builder) {
+        // remove extra comma if needed
+        if (builder.length() > 1) {
+            builder.deleteCharAt(builder.length() - 1);
+        }
+        builder.append("}");
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index d76c421..0ff877b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -142,4 +142,11 @@
      */
     String getActiveDataverseName(String dataverse);
 
+    /**
+     * Gets the execution plans that are generated during query compilation
+     *
+     * @return the executions plans
+     */
+    ExecutionPlans getExecutionPlans();
+
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 6ab76fa..0c46383 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -35,14 +35,13 @@
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -100,7 +99,6 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
@@ -151,6 +149,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 
@@ -192,20 +191,21 @@
     @Override
     public ILogicalPlan translateLoad(ICompiledDmlStatement stmt) throws AlgebricksException {
         CompiledLoadFromFileStatement clffs = (CompiledLoadFromFileStatement) stmt;
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         Dataset dataset = metadataProvider.findDataset(clffs.getDataverseName(), clffs.getDatasetName());
         if (dataset == null) {
             // This would never happen since we check for this in AqlTranslator
-            throw new AlgebricksException(
-                    "Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, clffs.getDatasetName(),
+                    clffs.getDataverseName());
         }
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
         IAType metaItemType =
                 metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         DatasetDataSource targetDatasource =
-                validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+                validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName(), sourceLoc);
         List<List<String>> partitionKeys = targetDatasource.getDataset().getPrimaryKeys();
         if (dataset.hasMetaPart()) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     dataset.getDatasetName() + ": load dataset is not supported on Datasets with Meta records");
         }
 
@@ -213,7 +213,7 @@
         try {
             lds = new LoadableDataSource(dataset, itemType, metaItemType, clffs.getAdapter(), clffs.getProperties());
         } catch (IOException e) {
-            throw new AlgebricksException(e);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, e.toString(), e);
         }
 
         // etsOp is a dummy input operator used to keep the compiler happy. it
@@ -229,7 +229,11 @@
         // Create a scan operator and make the empty tuple source its input
         DataSourceScanOperator dssOp = new DataSourceScanOperator(payloadVars, lds);
         dssOp.getInputs().add(new MutableObject<>(etsOp));
-        ILogicalExpression payloadExpr = new VariableReferenceExpression(payloadVars.get(0));
+        dssOp.setSourceLocation(sourceLoc);
+
+        VariableReferenceExpression payloadExpr = new VariableReferenceExpression(payloadVars.get(0));
+        payloadExpr.setSourceLocation(sourceLoc);
+
         Mutable<ILogicalExpression> payloadRef = new MutableObject<>(payloadExpr);
 
         // Creating the assign to extract the PK out of the record
@@ -239,11 +243,12 @@
         LogicalVariable payloadVar = payloadVars.get(0);
         for (List<String> keyFieldName : partitionKeys) {
             PlanTranslationUtil.prepareVarAndExpression(keyFieldName, payloadVar, pkVars, pkExprs, varRefsForLoading,
-                    context);
+                    context, sourceLoc);
         }
 
         AssignOperator assign = new AssignOperator(pkVars, pkExprs);
         assign.getInputs().add(new MutableObject<>(dssOp));
+        assign.setSourceLocation(sourceLoc);
 
         // If the input is pre-sorted, we set the ordering property explicitly in the
         // assign
@@ -265,14 +270,16 @@
             additionalFilteringAssignExpressions = new ArrayList<>();
             additionalFilteringExpressions = new ArrayList<>();
             PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, payloadVar, additionalFilteringVars,
-                    additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
+                    additionalFilteringAssignExpressions, additionalFilteringExpressions, context, sourceLoc);
             additionalFilteringAssign =
                     new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
+            additionalFilteringAssign.setSourceLocation(sourceLoc);
         }
 
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, payloadRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, true);
         insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+        insertOp.setSourceLocation(sourceLoc);
 
         if (additionalFilteringAssign != null) {
             additionalFilteringAssign.getInputs().add(new MutableObject<>(assign));
@@ -281,8 +288,9 @@
             insertOp.getInputs().add(new MutableObject<>(assign));
         }
 
-        ILogicalOperator leafOperator = new SinkOperator();
+        SinkOperator leafOperator = new SinkOperator();
         leafOperator.getInputs().add(new MutableObject<>(insertOp));
+        leafOperator.setSourceLocation(sourceLoc);
         return new ALogicalPlanImpl(new MutableObject<>(leafOperator));
     }
 
@@ -298,6 +306,7 @@
         if (baseOp != null) {
             base = new MutableObject<>(baseOp);
         }
+        SourceLocation sourceLoc = expr.getSourceLocation();
         Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, base);
         ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<>();
         ILogicalOperator topOp = p.first;
@@ -314,10 +323,13 @@
             metadataProvider.setOutputFile(outputFileSplit);
 
             List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<>(1);
-            writeExprList.add(new MutableObject<>(new VariableReferenceExpression(resVar)));
+            VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+            resVarRef.setSourceLocation(sourceLoc);
+            writeExprList.add(new MutableObject<>(resVarRef));
             ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
             ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
             DistributeResultOperator newTop = new DistributeResultOperator(writeExprList, sink);
+            newTop.setSourceLocation(sourceLoc);
             newTop.getInputs().add(new MutableObject<>(topOp));
             topOp = newTop;
 
@@ -337,17 +349,23 @@
              * This assign adds a marker function collection-to-sequence: if the input is a
              * singleton collection, unnest it; otherwise do nothing.
              */
-            AssignOperator assignCollectionToSequence = new AssignOperator(seqVar,
-                    new MutableObject<>(new ScalarFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.COLLECTION_TO_SEQUENCE),
-                            new MutableObject<>(new VariableReferenceExpression(resVar)))));
+            VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+            resVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression collectionToSequenceExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.COLLECTION_TO_SEQUENCE),
+                    new MutableObject<>(resVarRef));
+            collectionToSequenceExpr.setSourceLocation(sourceLoc);
+            AssignOperator assignCollectionToSequence =
+                    new AssignOperator(seqVar, new MutableObject<>(collectionToSequenceExpr));
+            assignCollectionToSequence.setSourceLocation(sourceLoc);
+
             assignCollectionToSequence.getInputs().add(new MutableObject<>(topOp.getInputs().get(0).getValue()));
             topOp.getInputs().get(0).setValue(assignCollectionToSequence);
             ProjectOperator projectOperator = (ProjectOperator) topOp;
             projectOperator.getVariables().set(0, seqVar);
             resVar = seqVar;
             DatasetDataSource targetDatasource =
-                    validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+                    validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName(), sourceLoc);
             List<Integer> keySourceIndicator =
                     ((InternalDatasetDetails) targetDatasource.getDataset().getDatasetDetails())
                             .getKeySourceIndicator();
@@ -360,15 +378,16 @@
                 if (keySourceIndicator == null || keySourceIndicator.get(i).intValue() == 0) {
                     // record part
                     PlanTranslationUtil.prepareVarAndExpression(partitionKeys.get(i), resVar, vars, exprs,
-                            varRefsForLoading, context);
+                            varRefsForLoading, context, sourceLoc);
                 } else {
                     // meta part
                     PlanTranslationUtil.prepareMetaKeyAccessExpression(partitionKeys.get(i), unnestVar, exprs, vars,
-                            varRefsForLoading, context);
+                            varRefsForLoading, context, sourceLoc);
                 }
             }
 
             AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.setSourceLocation(sourceLoc);
             List<String> additionalFilteringField = DatasetUtil.getFilterField(targetDatasource.getDataset());
             List<LogicalVariable> additionalFilteringVars;
             List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
@@ -380,17 +399,20 @@
                 additionalFilteringExpressions = new ArrayList<>();
 
                 PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, resVar, additionalFilteringVars,
-                        additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
+                        additionalFilteringAssignExpressions, additionalFilteringExpressions, context, sourceLoc);
 
                 additionalFilteringAssign =
                         new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
                 additionalFilteringAssign.getInputs().add(new MutableObject<>(topOp));
+                additionalFilteringAssign.setSourceLocation(sourceLoc);
                 assign.getInputs().add(new MutableObject<>(additionalFilteringAssign));
             } else {
                 assign.getInputs().add(new MutableObject<>(topOp));
             }
 
-            Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(resVar));
+            VariableReferenceExpression resVarRef2 = new VariableReferenceExpression(resVar);
+            resVarRef2.setSourceLocation(sourceLoc);
+            Mutable<ILogicalExpression> varRef = new MutableObject<>(resVarRef2);
             ILogicalOperator leafOperator;
             switch (stmt.getKind()) {
                 case INSERT:
@@ -404,10 +426,11 @@
                     break;
                 case DELETE:
                     leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
-                            additionalFilteringExpressions, assign);
+                            additionalFilteringExpressions, assign, stmt);
                     break;
                 default:
-                    throw new AlgebricksException("Unsupported statement kind " + stmt.getKind());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unsupported statement kind " + stmt.getKind());
             }
             topOp = leafOperator;
         }
@@ -419,18 +442,22 @@
 
     private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
-            List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
-            throws AlgebricksException {
+            List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
+            ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (targetDatasource.getDataset().hasMetaPart()) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": delete from dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": delete from dataset is not supported on Datasets with Meta records");
         }
         InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
         deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         deleteOp.getInputs().add(new MutableObject<>(assign));
-        ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(true));
+        deleteOp.setSourceLocation(sourceLoc);
+        DelegateOperator leafOperator = new DelegateOperator(new CommitOperator(true));
         leafOperator.getInputs().add(new MutableObject<>(deleteOp));
+        leafOperator.setSourceLocation(sourceLoc);
         return leafOperator;
     }
 
@@ -440,9 +467,11 @@
             List<String> additionalFilteringField, LogicalVariable unnestVar, ILogicalOperator topOp,
             List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (!targetDatasource.getDataset().allow(topOp, DatasetUtil.OP_UPSERT)) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": upsert into dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": upsert into dataset is not supported on Datasets with Meta records");
         }
         ProjectOperator project = (ProjectOperator) topOp;
         CompiledUpsertStatement compiledUpsert = (CompiledUpsertStatement) stmt;
@@ -451,7 +480,8 @@
         ILogicalOperator rootOperator;
         if (targetDatasource.getDataset().hasMetaPart()) {
             if (returnExpression != null) {
-                throw new AlgebricksException("Returning not allowed on datasets with Meta records");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Returning not allowed on datasets with Meta records");
             }
             AssignOperator metaAndKeysAssign;
             List<LogicalVariable> metaAndKeysVars;
@@ -461,12 +491,17 @@
             metaAndKeysExprs = new ArrayList<>();
             // add the meta function
             IFunctionInfo finfoMeta = FunctionUtil.getFunctionInfo(BuiltinFunctions.META);
-            ScalarFunctionCallExpression metaFunction = new ScalarFunctionCallExpression(finfoMeta,
-                    new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+            VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+            unnestVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression metaFunction =
+                    new ScalarFunctionCallExpression(finfoMeta, new MutableObject<>(unnestVarRef));
+            metaFunction.setSourceLocation(sourceLoc);
             // create assign for the meta part
             LogicalVariable metaVar = context.newVar();
             metaExpSingletonList = new ArrayList<>(1);
-            metaExpSingletonList.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
+            VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+            metaVarRef.setSourceLocation(sourceLoc);
+            metaExpSingletonList.add(new MutableObject<>(metaVarRef));
             metaAndKeysVars.add(metaVar);
             metaAndKeysExprs.add(new MutableObject<>(metaFunction));
             project.getVariables().add(metaVar);
@@ -488,6 +523,7 @@
             // Create and add a new variable used for representing the original record
             upsertOp.setPrevRecordVar(context.newVar());
             upsertOp.setPrevRecordType(targetDatasource.getItemType());
+            upsertOp.setSourceLocation(sourceLoc);
             if (targetDatasource.getDataset().hasMetaPart()) {
                 List<LogicalVariable> metaVars = new ArrayList<>();
                 metaVars.add(context.newVar());
@@ -509,6 +545,7 @@
             }
             metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
             metaAndKeysAssign.getInputs().add(topOp.getInputs().get(0));
+            metaAndKeysAssign.setSourceLocation(sourceLoc);
             topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
             upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         } else {
@@ -516,6 +553,7 @@
                     InsertDeleteUpsertOperator.Kind.UPSERT, false);
             upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
             upsertOp.getInputs().add(new MutableObject<>(assign));
+            upsertOp.setSourceLocation(sourceLoc);
             // Create and add a new variable used for representing the original record
             ARecordType recordType = (ARecordType) targetDatasource.getItemType();
             upsertOp.setPrevRecordVar(context.newVar());
@@ -525,34 +563,38 @@
                 upsertOp.setPrevFilterType(recordType.getFieldType(additionalFilteringField.get(0)));
             }
         }
-        rootOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
-        rootOperator.getInputs().add(new MutableObject<>(upsertOp));
+        DelegateOperator delegateOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
+        delegateOperator.getInputs().add(new MutableObject<>(upsertOp));
+        delegateOperator.setSourceLocation(sourceLoc);
+        rootOperator = delegateOperator;
 
         // Compiles the return expression.
         return processReturningExpression(rootOperator, upsertOp, compiledUpsert);
-
     }
 
     private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (targetDatasource.getDataset().hasMetaPart()) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": insert into dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": insert into dataset is not supported on Datasets with Meta records");
         }
         // Adds the insert operator.
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
         insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         insertOp.getInputs().add(new MutableObject<>(assign));
+        insertOp.setSourceLocation(sourceLoc);
 
         // Adds the commit operator.
         CompiledInsertStatement compiledInsert = (CompiledInsertStatement) stmt;
         Expression returnExpression = compiledInsert.getReturnExpression();
-        ILogicalOperator rootOperator =
-                new DelegateOperator(new CommitOperator(returnExpression == null ? true : false));
+        DelegateOperator rootOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
         rootOperator.getInputs().add(new MutableObject<>(insertOp));
+        rootOperator.setSourceLocation(sourceLoc);
 
         // Compiles the return expression.
         return processReturningExpression(rootOperator, insertOp, compiledInsert);
@@ -566,38 +608,51 @@
         if (returnExpression == null) {
             return inputOperator;
         }
-        ILogicalOperator rootOperator = inputOperator;
+        SourceLocation sourceLoc = compiledInsert.getSourceLocation();
+
+        //Create an assign operator that makes the variable used by the return expression
+        LogicalVariable insertedVar = context.newVar();
+        AssignOperator insertedVarAssignOperator =
+                new AssignOperator(insertedVar, new MutableObject<>(insertOp.getPayloadExpression().getValue()));
+        insertedVarAssignOperator.getInputs().add(insertOp.getInputs().get(0));
+        insertedVarAssignOperator.setSourceLocation(sourceLoc);
+        insertOp.getInputs().set(0, new MutableObject<>(insertedVarAssignOperator));
 
         // Makes the id of the insert var point to the record variable.
         context.newVarFromExpression(compiledInsert.getVar());
-        context.setVar(compiledInsert.getVar(),
-                ((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
-                langExprToAlgExpression(returnExpression, new MutableObject<>(rootOperator));
+        context.setVar(compiledInsert.getVar(), insertedVar);
 
-        // Adds an assign operator for the returning expression.
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
+                langExprToAlgExpression(returnExpression, new MutableObject<>(inputOperator));
+
+        // Adds an assign operator for the result of the returning expression.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator assignOperator = new AssignOperator(resultVar, new MutableObject<>(p.first));
-        assignOperator.getInputs().add(p.second);
+        AssignOperator createResultAssignOperator = new AssignOperator(resultVar, new MutableObject<>(p.first));
+        createResultAssignOperator.getInputs().add(p.second);
+        createResultAssignOperator.setSourceLocation(sourceLoc);
 
         // Adds a distribute result operator.
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         expressions.add(new MutableObject<>(new VariableReferenceExpression(resultVar)));
         ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
         ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
-        rootOperator = new DistributeResultOperator(expressions, sink);
-        rootOperator.getInputs().add(new MutableObject<>(assignOperator));
-        return rootOperator;
+        DistributeResultOperator distResultOperator = new DistributeResultOperator(expressions, sink);
+        distResultOperator.getInputs().add(new MutableObject<>(createResultAssignOperator));
+
+        distResultOperator.setSourceLocation(sourceLoc);
+        return distResultOperator;
     }
 
     private DatasetDataSource validateDatasetInfo(MetadataProvider metadataProvider, String dataverseName,
-            String datasetName) throws AlgebricksException {
+            String datasetName, SourceLocation sourceLoc) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Cannot find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                    dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Cannot write output to an external dataset.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Cannot write output to an external dataset.");
         }
         DataSourceId sourceId = new DataSourceId(dataverseName, datasetName);
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
@@ -620,56 +675,69 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(LetClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         LogicalVariable v;
-        ILogicalOperator returnedOp;
-        if (lc.getBindingExpr().getKind() == Kind.VARIABLE_EXPRESSION) {
+        AssignOperator returnedOp;
+        Expression bindingExpr = lc.getBindingExpr();
+        SourceLocation sourceLoc = bindingExpr.getSourceLocation();
+        if (bindingExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
             v = context.newVarFromExpression(lc.getVarExpr());
-            LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
-            returnedOp = new AssignOperator(v, new MutableObject<>(new VariableReferenceExpression(prev)));
+            LogicalVariable prevVar = context.getVar(((VariableExpr) bindingExpr).getVar().getId());
+            VariableReferenceExpression prevVarRef = new VariableReferenceExpression(prevVar);
+            prevVarRef.setSourceLocation(sourceLoc);
+            returnedOp = new AssignOperator(v, new MutableObject<>(prevVarRef));
             returnedOp.getInputs().add(tupSource);
+            returnedOp.setSourceLocation(sourceLoc);
         } else {
             v = context.newVarFromExpression(lc.getVarExpr());
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
-                    langExprToAlgExpression(lc.getBindingExpr(), tupSource);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(bindingExpr, tupSource);
             returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
             returnedOp.getInputs().add(eo.second);
+            returnedOp.setSourceLocation(sourceLoc);
         }
+
         return new Pair<>(returnedOp, v);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = fa.getSourceLocation();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(fa.getExpr(), tupSource);
         LogicalVariable v = context.newVarFromExpression(fa);
         AbstractFunctionCallExpression fldAccess =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(p.first));
         ILogicalExpression faExpr =
                 new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
         fldAccess.getArguments().add(new MutableObject<>(faExpr));
         AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
         a.getInputs().add(p.second);
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, v);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = ia.getSourceLocation();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(ia.getExpr(), tupSource);
         LogicalVariable v = context.newVar();
         AbstractFunctionCallExpression f;
         if (ia.isAny()) {
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.ANY_COLLECTION_MEMBER));
             f.getArguments().add(new MutableObject<>(p.first));
+            f.setSourceLocation(sourceLoc);
         } else {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair =
                     langExprToAlgExpression(ia.getIndexExpr(), tupSource);
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM));
             f.getArguments().add(new MutableObject<>(p.first));
             f.getArguments().add(new MutableObject<>(indexPair.first));
+            f.setSourceLocation(sourceLoc);
         }
         AssignOperator a = new AssignOperator(v, new MutableObject<>(f));
         a.getInputs().add(p.second);
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, v);
     }
 
@@ -684,8 +752,11 @@
         for (Expression expr : fcall.getExprList()) {
             switch (expr.getKind()) {
                 case VARIABLE_EXPRESSION:
-                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
-                    args.add(new MutableObject<>(new VariableReferenceExpression(var)));
+                    VariableExpr varExpr = (VariableExpr) expr;
+                    LogicalVariable var = context.getVar(varExpr.getVar().getId());
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                    varRef.setSourceLocation(varExpr.getSourceLocation());
+                    args.add(new MutableObject<>(varRef));
                     break;
                 case LITERAL_EXPRESSION:
                     LiteralExpr val = (LiteralExpr) expr;
@@ -703,17 +774,15 @@
             }
         }
 
+        SourceLocation sourceLoc = fcall.getSourceLocation();
         AbstractFunctionCallExpression f;
-        try {
-            if ((f = lookupUserDefinedFunction(signature, args)) == null) {
-                f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args);
-            }
-        } catch (AlgebricksException e) {
-            throw new CompilationException(e);
+        if ((f = lookupUserDefinedFunction(signature, args, sourceLoc)) == null) {
+            f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args, sourceLoc);
         }
 
         if (f == null) {
-            throw new CompilationException(" Unknown function " + signature.getName() + "@" + signature.getArity());
+            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc,
+                    signature.getName() + "@" + signature.getArity());
         }
 
         // Put hints into function call expr.
@@ -727,37 +796,45 @@
         if (topOp != null) {
             op.getInputs().add(topOp);
         }
+        op.setSourceLocation(sourceLoc);
 
         return new Pair<>(op, v);
     }
 
     private AbstractFunctionCallExpression lookupUserDefinedFunction(FunctionSignature signature,
-            List<Mutable<ILogicalExpression>> args) throws AlgebricksException {
-        if (signature.getNamespace() == null) {
-            return null;
+            List<Mutable<ILogicalExpression>> args, SourceLocation sourceLoc) throws CompilationException {
+        try {
+            if (signature.getNamespace() == null) {
+                return null;
+            }
+            Function function =
+                    MetadataManager.INSTANCE.getFunction(metadataProvider.getMetadataTxnContext(), signature);
+            if (function == null) {
+                return null;
+            }
+            AbstractFunctionCallExpression f;
+            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
+                IFunctionInfo finfo = ExternalFunctionCompilerUtil
+                        .getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
+                f = new ScalarFunctionCallExpression(finfo, args);
+                f.setSourceLocation(sourceLoc);
+            } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
+                    || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
+                IFunctionInfo finfo = FunctionUtil.getFunctionInfo(signature);
+                f = new ScalarFunctionCallExpression(finfo, args);
+                f.setSourceLocation(sourceLoc);
+            } else {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        " User defined functions written in " + function.getLanguage() + " are not supported");
+            }
+            return f;
+        } catch (AlgebricksException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage(), e);
         }
-        Function function = MetadataManager.INSTANCE.getFunction(metadataProvider.getMetadataTxnContext(), signature);
-        if (function == null) {
-            return null;
-        }
-        AbstractFunctionCallExpression f;
-        if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
-            IFunctionInfo finfo = ExternalFunctionCompilerUtil
-                    .getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
-            f = new ScalarFunctionCallExpression(finfo, args);
-        } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
-                || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
-            IFunctionInfo finfo = FunctionUtil.getFunctionInfo(signature);
-            f = new ScalarFunctionCallExpression(finfo, args);
-        } else {
-            throw new MetadataException(
-                    " User defined functions written in " + function.getLanguage() + " are not supported");
-        }
-        return f;
     }
 
     private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
-            List<Mutable<ILogicalExpression>> args) {
+            List<Mutable<ILogicalExpression>> args, SourceLocation sourceLoc) {
         AbstractFunctionCallExpression f;
         FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
         FunctionInfo afi = BuiltinFunctions.lookupFunction(fi);
@@ -782,6 +859,7 @@
         } else {
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
         }
+        f.setSourceLocation(sourceLoc);
         return f;
     }
 
@@ -793,8 +871,8 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = gc.getSourceLocation();
         Mutable<ILogicalOperator> topOp = tupSource;
-
         LogicalVariable groupRecordVar = null;
         if (gc.hasGroupVar()) {
             List<Pair<Expression, Identifier>> groupFieldList = gc.getGroupFieldList();
@@ -807,13 +885,16 @@
                 ILogicalExpression groupFieldExpr = langExprToAlgExpression(groupField.first, topOp).first;
                 groupRecordConstructorArgList.add(new MutableObject<>(groupFieldExpr));
             }
-            MutableObject<ILogicalExpression> groupRecordConstr = new MutableObject<>(new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression groupRecordConstr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR),
-                    groupRecordConstructorArgList));
+                    groupRecordConstructorArgList);
+            groupRecordConstr.setSourceLocation(sourceLoc);
 
             groupRecordVar = context.newVar();
-            AssignOperator groupRecordVarAssignOp = new AssignOperator(groupRecordVar, groupRecordConstr);
+            AssignOperator groupRecordVarAssignOp =
+                    new AssignOperator(groupRecordVar, new MutableObject<>(groupRecordConstr));
             groupRecordVarAssignOp.getInputs().add(topOp);
+            groupRecordVarAssignOp.setSourceLocation(sourceLoc);
             topOp = new MutableObject<>(groupRecordVarAssignOp);
         }
 
@@ -838,9 +919,12 @@
         if (gc.hasGroupVar()) {
             VariableExpr groupVar = gc.getGroupVar();
             LogicalVariable groupLogicalVar = context.newVar();
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(gOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            VariableReferenceExpression groupRecordVarRef = new VariableReferenceExpression(groupRecordVar);
+            groupRecordVarRef.setSourceLocation(sourceLoc);
             ILogicalPlan nestedPlan = createNestedPlanWithAggregate(groupLogicalVar, BuiltinFunctions.LISTIFY,
-                    new VariableReferenceExpression(groupRecordVar),
-                    new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(gOp))));
+                    groupRecordVarRef, new MutableObject<>(ntsOp));
             gOp.getNestedPlans().add(nestedPlan);
             context.setVar(groupVar, groupLogicalVar);
         }
@@ -849,8 +933,10 @@
             for (Entry<Expression, VariableExpr> entry : gc.getWithVarMap().entrySet()) {
                 VariableExpr withVar = entry.getValue();
                 Expression withExpr = entry.getKey();
-                Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput = langExprToAlgExpression(withExpr,
-                        new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(gOp))));
+                NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(gOp));
+                ntsOp.setSourceLocation(sourceLoc);
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput =
+                        langExprToAlgExpression(withExpr, new MutableObject<>(ntsOp));
                 LogicalVariable withLogicalVar = context.newVar();
                 ILogicalPlan nestedPlan = createNestedPlanWithAggregate(withLogicalVar, BuiltinFunctions.LISTIFY,
                         listifyInput.first, listifyInput.second);
@@ -861,16 +947,20 @@
 
         gOp.setGroupAll(gc.isGroupAll());
         gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        gOp.setSourceLocation(sourceLoc);
         return new Pair<>(gOp, null);
     }
 
     private ILogicalPlan createNestedPlanWithAggregate(LogicalVariable aggOutputVar, FunctionIdentifier aggFunc,
             ILogicalExpression aggFnInput, Mutable<ILogicalOperator> aggOpInput) {
+        SourceLocation sourceLoc = aggFnInput.getSourceLocation();
         AggregateFunctionCallExpression aggFnCall = BuiltinFunctions.makeAggregateFunctionExpression(aggFunc,
                 mkSingletonArrayList(new MutableObject<>(aggFnInput)));
+        aggFnCall.setSourceLocation(sourceLoc);
         AggregateOperator aggOp = new AggregateOperator(mkSingletonArrayList(aggOutputVar),
                 mkSingletonArrayList(new MutableObject<>(aggFnCall)));
         aggOp.getInputs().add(aggOpInput);
+        aggOp.setSourceLocation(sourceLoc);
         return new ALogicalPlanImpl(new MutableObject<>(aggOp));
     }
 
@@ -886,45 +976,70 @@
         // Similarly, we create one subplan for the "else" branch, in which the
         // selection is not(varCond).
         // Finally, we select the desired result.
-        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        Expression condExpr = ifexpr.getCondExpr();
+        Expression thenExpr = ifexpr.getThenExpr();
+        Expression elseExpr = ifexpr.getElseExpr();
+
+        Pair<ILogicalOperator, LogicalVariable> pCond = condExpr.accept(this, tupSource);
         LogicalVariable varCond = pCond.second;
 
         // Creates a subplan for the "then" branch.
-        Pair<ILogicalOperator, LogicalVariable> opAndVarForThen = constructSubplanOperatorForBranch(pCond.first,
-                new MutableObject<>(new VariableReferenceExpression(varCond)), ifexpr.getThenExpr());
+        VariableReferenceExpression varCondRef1 = new VariableReferenceExpression(varCond);
+        varCondRef1.setSourceLocation(condExpr.getSourceLocation());
+
+        Pair<ILogicalOperator, LogicalVariable> opAndVarForThen =
+                constructSubplanOperatorForBranch(pCond.first, new MutableObject<>(varCondRef1), thenExpr);
 
         // Creates a subplan for the "else" branch.
-        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-                Collections.singletonList(generateAndNotIsUnknownWrap(new VariableReferenceExpression(varCond))));
-        Pair<ILogicalOperator, LogicalVariable> opAndVarForElse = constructSubplanOperatorForBranch(
-                opAndVarForThen.first, new MutableObject<>(notVarCond), ifexpr.getElseExpr());
+        VariableReferenceExpression varCondRef2 = new VariableReferenceExpression(varCond);
+        varCondRef2.setSourceLocation(condExpr.getSourceLocation());
+        AbstractFunctionCallExpression notVarCond =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+                        Collections.singletonList(generateAndNotIsUnknownWrap(varCondRef2)));
+        notVarCond.setSourceLocation(condExpr.getSourceLocation());
+
+        Pair<ILogicalOperator, LogicalVariable> opAndVarForElse =
+                constructSubplanOperatorForBranch(opAndVarForThen.first, new MutableObject<>(notVarCond), elseExpr);
 
         // Uses switch-case function to select the results of two branches.
         LogicalVariable selectVar = context.newVar();
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(varCond)));
+        VariableReferenceExpression varCondRef3 = new VariableReferenceExpression(varCond);
+        varCondRef3.setSourceLocation(condExpr.getSourceLocation());
+        VariableReferenceExpression varThenRef = new VariableReferenceExpression(opAndVarForThen.second);
+        varThenRef.setSourceLocation(thenExpr.getSourceLocation());
+        VariableReferenceExpression varElseRef = new VariableReferenceExpression(opAndVarForElse.second);
+        varElseRef.setSourceLocation(elseExpr.getSourceLocation());
+        arguments.add(new MutableObject<>(varCondRef3));
         arguments.add(new MutableObject<>(ConstantExpression.TRUE));
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForThen.second)));
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForElse.second)));
+        arguments.add(new MutableObject<>(varThenRef));
+        arguments.add(new MutableObject<>(varElseRef));
         AbstractFunctionCallExpression swithCaseExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SWITCH_CASE), arguments);
+        swithCaseExpr.setSourceLocation(ifexpr.getSourceLocation());
         AssignOperator assignOp = new AssignOperator(selectVar, new MutableObject<>(swithCaseExpr));
         assignOp.getInputs().add(new MutableObject<>(opAndVarForElse.first));
+        assignOp.setSourceLocation(ifexpr.getSourceLocation());
 
         // Unnests the selected ("if" or "else") result.
         LogicalVariable unnestVar = context.newVar();
-        UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                new MutableObject<>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), Collections
-                                .singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
+        VariableReferenceExpression selectVarRef = new VariableReferenceExpression(selectVar);
+        selectVarRef.setSourceLocation(ifexpr.getSourceLocation());
+        UnnestingFunctionCallExpression scanCollExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        Collections.singletonList(new MutableObject<>(selectVarRef)));
+        scanCollExpr.setSourceLocation(ifexpr.getSourceLocation());
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollExpr));
         unnestOp.getInputs().add(new MutableObject<>(assignOp));
+        unnestOp.setSourceLocation(ifexpr.getSourceLocation());
 
         // Produces the final result.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator finalAssignOp =
-                new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(ifexpr.getSourceLocation());
+        AssignOperator finalAssignOp = new AssignOperator(resultVar, new MutableObject<>(unnestVarRef));
         finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
+        finalAssignOp.setSourceLocation(ifexpr.getSourceLocation());
         return new Pair<>(finalAssignOp, resultVar);
     }
 
@@ -933,6 +1048,7 @@
         LogicalVariable var = context.newVar();
         AssignOperator a = new AssignOperator(var, new MutableObject<>(
                 new ConstantExpression(new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        a.setSourceLocation(l.getSourceLocation());
         if (tupSource != null) {
             a.getInputs().add(tupSource);
         }
@@ -953,16 +1069,16 @@
 
         Mutable<ILogicalOperator> topOp = tupSource;
 
-        ILogicalExpression currExpr = null;
+        SourceLocation sourceLoc = op.getSourceLocation();
+        AbstractFunctionCallExpression currExpr = null;
         for (int i = 0; i <= nOps; i++) {
-
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(exprs.get(i), topOp);
             topOp = p.second;
             ILogicalExpression e = p.first;
             // now look at the operator
             if (i < nOps) {
                 if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
+                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i), sourceLoc);
 
                     // chain the operators
                     if (i == 0) {
@@ -974,7 +1090,7 @@
                             c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                         }
                     } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
+                        currExpr.getArguments().add(new MutableObject<>(e));
                         c.getArguments().add(new MutableObject<>(currExpr));
                         currExpr = c;
                         if (i == 1 && op.isBroadcastOperand(i)) {
@@ -984,47 +1100,47 @@
                         }
                     }
                 } else {
-                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
+                    AbstractFunctionCallExpression f =
+                            createFunctionCallExpressionForBuiltinOperator(ops.get(i), sourceLoc);
                     if (i == 0) {
                         f.getArguments().add(new MutableObject<>(e));
                         currExpr = f;
                     } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
+                        currExpr.getArguments().add(new MutableObject<>(e));
                         f.getArguments().add(new MutableObject<>(currExpr));
                         currExpr = f;
                     }
                 }
             } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
+                currExpr.getArguments().add(new MutableObject<>(e));
                 if (i == 1 && op.isBroadcastOperand(i)) {
                     BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
                     bcast.setObject(BroadcastSide.RIGHT);
-                    ((AbstractFunctionCallExpression) currExpr).getAnnotations()
-                            .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                    currExpr.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                 }
             }
         }
 
         // Add hints as annotations.
-        if (op.hasHints() && (currExpr instanceof AbstractFunctionCallExpression)) {
-            AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
+        if (op.hasHints()) {
             for (IExpressionAnnotation hint : op.getHints()) {
-                currFuncExpr.getAnnotations().put(hint, hint);
+                currExpr.getAnnotations().put(hint, hint);
             }
         }
 
         LogicalVariable assignedVar = context.newVar();
         AssignOperator a = new AssignOperator(assignedVar, new MutableObject<>(currExpr));
-
         a.getInputs().add(topOp);
-
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, assignedVar);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = oc.getSourceLocation();
         OrderOperator ord = new OrderOperator();
+        ord.setSourceLocation(sourceLoc);
         Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
         Mutable<ILogicalOperator> topOp = tupSource;
         for (Expression e : oc.getOrderbyList()) {
@@ -1053,16 +1169,19 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = qe.getSourceLocation();
         Mutable<ILogicalOperator> topOp = tupSource;
 
         ILogicalOperator firstOp = null;
         Mutable<ILogicalOperator> lastOp = null;
 
         for (QuantifiedPair qt : qe.getQuantifiedList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(qt.getExpr(), topOp);
+            Expression expr = qt.getExpr();
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(expr, topOp);
             topOp = eo1.second;
             LogicalVariable uVar = context.newVarFromExpression(qt.getVarExpr());
-            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<>(makeUnnestExpression(eo1.first)));
+            UnnestOperator u = new UnnestOperator(uVar, new MutableObject<>(makeUnnestExpression(eo1.first)));
+            u.setSourceLocation(expr.getSourceLocation());
 
             if (firstOp == null) {
                 firstOp = u;
@@ -1085,8 +1204,10 @@
         if (qe.getQuantifier() == Quantifier.SOME) {
             s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
             s.getInputs().add(eo2.second);
+            s.setSourceLocation(sourceLoc);
             fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM,
                     new ArrayList<>());
+            fAgg.setSourceLocation(sourceLoc);
         } else { // EVERY
             // look for input items that do not satisfy the condition, if none found then return true
             // when inverting the condition account for NULL/MISSING by replacing them with FALSE
@@ -1098,18 +1219,25 @@
                     .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE))));
 
             List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
-            notArgs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs)));
+            ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs);
+            ifMissinOrNullExpr.setSourceLocation(sourceLoc);
+            notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
 
-            s = new SelectOperator(new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs)), false, null);
+            ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
+            notExpr.setSourceLocation(sourceLoc);
+            s = new SelectOperator(new MutableObject<>(notExpr), false, null);
             s.getInputs().add(eo2.second);
+            s.setSourceLocation(sourceLoc);
             fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM, new ArrayList<>());
+            fAgg.setSourceLocation(sourceLoc);
         }
         LogicalVariable qeVar = context.newVar();
-        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List) mkSingletonArrayList(new MutableObject<>(fAgg)));
+        AggregateOperator a =
+                new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
         a.getInputs().add(new MutableObject<>(s));
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, qeVar);
     }
 
@@ -1124,8 +1252,10 @@
             throws CompilationException {
         AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        f.setSourceLocation(rc.getSourceLocation());
         LogicalVariable v1 = context.newVar();
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(f));
+        a.setSourceLocation(rc.getSourceLocation());
         Mutable<ILogicalOperator> topOp = tupSource;
         for (FieldBinding fb : rc.getFbList()) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(fb.getLeftExpr(), topOp);
@@ -1142,11 +1272,14 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(ListConstructor lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = lc.getSourceLocation();
         FunctionIdentifier fid = (lc.getType() == ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)
                 ? BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR : BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
         AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        f.setSourceLocation(sourceLoc);
         LogicalVariable v1 = context.newVar();
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(f));
+        a.setSourceLocation(sourceLoc);
         Mutable<ILogicalOperator> topOp = tupSource;
         for (Expression expr : lc.getExprList()) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
@@ -1160,6 +1293,7 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = u.getSourceLocation();
         Expression expr = u.getExpr();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, tupSource);
         LogicalVariable v1 = context.newVar();
@@ -1167,21 +1301,25 @@
         switch (u.getExprType()) {
             case POSITIVE:
                 a = new AssignOperator(v1, new MutableObject<>(eo.first));
+                a.setSourceLocation(sourceLoc);
                 break;
             case NEGATIVE:
                 AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.NUMERIC_UNARY_MINUS));
+                m.setSourceLocation(sourceLoc);
                 m.getArguments().add(new MutableObject<>(eo.first));
                 a = new AssignOperator(v1, new MutableObject<>(m));
+                a.setSourceLocation(sourceLoc);
                 break;
             case EXISTS:
-                a = processExists(eo.first, v1, false);
+                a = processExists(eo.first, v1, false, sourceLoc);
                 break;
             case NOT_EXISTS:
-                a = processExists(eo.first, v1, true);
+                a = processExists(eo.first, v1, true, sourceLoc);
                 break;
             default:
-                throw new CompilationException("Unsupported operator: " + u);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported operator: " + u.getExprType());
         }
         a.getInputs().add(eo.second);
         return new Pair<>(a, v1);
@@ -1189,11 +1327,15 @@
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+        SourceLocation sourceLoc = v.getSourceLocation();
         // Should we ever get to this method?
         LogicalVariable var = context.newVar();
         LogicalVariable oldV = context.getVar(v.getVar().getId());
-        AssignOperator a = new AssignOperator(var, new MutableObject<>(new VariableReferenceExpression(oldV)));
+        VariableReferenceExpression oldVRef = new VariableReferenceExpression(oldV);
+        oldVRef.setSourceLocation(sourceLoc);
+        AssignOperator a = new AssignOperator(var, new MutableObject<>(oldVRef));
         a.getInputs().add(tupSource);
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, var);
     }
 
@@ -1203,12 +1345,14 @@
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(w.getWhereExpr(), tupSource);
         SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
         s.getInputs().add(p.second);
+        s.setSourceLocation(w.getSourceLocation());
         return new Pair<>(s, null);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(LimitClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = lc.getSourceLocation();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
         LimitOperator opLim;
         Expression offset = lc.getOffset();
@@ -1216,17 +1360,21 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = langExprToAlgExpression(offset, p1.second);
             opLim = new LimitOperator(p1.first, p2.first);
             opLim.getInputs().add(p2.second);
+            opLim.setSourceLocation(sourceLoc);
         } else {
             opLim = new LimitOperator(p1.first);
             opLim.getInputs().add(p1.second);
+            opLim.setSourceLocation(sourceLoc);
         }
         return new Pair<>(opLim, null);
     }
 
-    protected AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+    protected AbstractFunctionCallExpression createComparisonExpression(OperatorType t, SourceLocation sourceLoc) {
         FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
         IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
-        return new ScalarFunctionCallExpression(finfo);
+        ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(finfo);
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
     private static FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
@@ -1248,8 +1396,8 @@
         }
     }
 
-    protected AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
-            throws CompilationException {
+    protected AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
+            SourceLocation sourceLoc) throws CompilationException {
         FunctionIdentifier fid;
         switch (t) {
             case PLUS:
@@ -1261,17 +1409,17 @@
             case MUL:
                 fid = BuiltinFunctions.NUMERIC_MULTIPLY;
                 break;
-            case DIV:
+            case DIVIDE:
                 fid = BuiltinFunctions.NUMERIC_DIVIDE;
                 break;
+            case DIV:
+                fid = BuiltinFunctions.NUMERIC_DIV;
+                break;
             case MOD:
                 fid = BuiltinFunctions.NUMERIC_MOD;
                 break;
-            case IDIV:
-                fid = BuiltinFunctions.NUMERIC_IDIV;
-                break;
             case CARET:
-                fid = BuiltinFunctions.CARET;
+                fid = BuiltinFunctions.NUMERIC_POWER;
                 break;
             case AND:
                 fid = AlgebricksBuiltinFunctions.AND;
@@ -1283,9 +1431,12 @@
                 fid = BuiltinFunctions.FUZZY_EQ;
                 break;
             default:
-                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Operator " + t + " is not yet implemented");
         }
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
     private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
@@ -1298,6 +1449,7 @@
 
     protected Pair<ILogicalExpression, Mutable<ILogicalOperator>> langExprToAlgExpression(Expression expr,
             Mutable<ILogicalOperator> topOpRef) throws CompilationException {
+        SourceLocation sourceLoc = expr.getSourceLocation();
         switch (expr.getKind()) {
             case VARIABLE_EXPRESSION:
                 LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
@@ -1305,6 +1457,7 @@
                     throw new IllegalStateException(String.valueOf(expr));
                 }
                 VariableReferenceExpression ve = new VariableReferenceExpression(var);
+                ve.setSourceLocation(sourceLoc);
                 return new Pair<>(ve, topOpRef);
             case LITERAL_EXPRESSION:
                 LiteralExpr val = (LiteralExpr) expr;
@@ -1326,14 +1479,21 @@
                             rebindBottomOpRef(p.first, srcRef, topOpRef);
                         }
                         Mutable<ILogicalOperator> top2 = new MutableObject<>(p.first);
-                        return new Pair<>(new VariableReferenceExpression(p.second), top2);
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(p.second);
+                        varRef.setSourceLocation(sourceLoc);
+                        return new Pair<>(varRef, top2);
                     } else {
                         SubplanOperator s = new SubplanOperator();
                         s.getInputs().add(topOpRef);
-                        srcRef.setValue(new NestedTupleSourceOperator(new MutableObject<>(s)));
+                        s.setSourceLocation(sourceLoc);
+                        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(s));
+                        ntsOp.setSourceLocation(sourceLoc);
+                        srcRef.setValue(ntsOp);
                         Mutable<ILogicalOperator> planRoot = new MutableObject<>(p.first);
                         s.setRootOp(planRoot);
-                        return new Pair<>(new VariableReferenceExpression(p.second), new MutableObject<>(s));
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(p.second);
+                        varRef.setSourceLocation(sourceLoc);
+                        return new Pair<>(varRef, new MutableObject<>(s));
                     }
                 }
         }
@@ -1341,18 +1501,22 @@
 
     protected Pair<ILogicalOperator, LogicalVariable> aggListifyForSubquery(LogicalVariable var,
             Mutable<ILogicalOperator> opRef, boolean bProject) {
+        SourceLocation sourceLoc = opRef.getValue().getSourceLocation();
         AggregateFunctionCallExpression funAgg =
                 BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, new ArrayList<>());
         funAgg.getArguments().add(new MutableObject<>(new VariableReferenceExpression(var)));
+        funAgg.setSourceLocation(sourceLoc);
 
         LogicalVariable varListified = context.newSubplanOutputVar();
         AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
                 mkSingletonArrayList(new MutableObject<>(funAgg)));
         agg.getInputs().add(opRef);
+        agg.setSourceLocation(sourceLoc);
         ILogicalOperator res;
         if (bProject) {
             ProjectOperator pr = new ProjectOperator(varListified);
             pr.getInputs().add(new MutableObject<>(agg));
+            pr.setSourceLocation(sourceLoc);
             res = pr;
         } else {
             res = agg;
@@ -1362,6 +1526,7 @@
 
     protected Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
             Mutable<ILogicalOperator> tupSource) throws CompilationException {
+        SourceLocation sourceLoc = op.getSourceLocation();
         List<OperatorType> ops = op.getOpList();
         int nOps = ops.size();
 
@@ -1370,15 +1535,15 @@
         Mutable<ILogicalOperator> topOp = tupSource;
 
         OperatorType opLogical = ops.get(0);
-        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical, sourceLoc);
 
         for (int i = 0; i <= nOps; i++) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(exprs.get(i), topOp);
             topOp = p.second;
             // now look at the operator
             if (i < nOps && ops.get(i) != opLogical) {
-                throw new TranslationException(
-                        "Unexpected operator " + ops.get(i) + " in an OperatorExpr starting with " + opLogical);
+                throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_OPERATOR, sourceLoc, ops.get(i),
+                        opLogical);
             }
             f.getArguments().add(new MutableObject<>(p.first));
         }
@@ -1386,6 +1551,7 @@
         LogicalVariable assignedVar = context.newVar();
         AssignOperator a = new AssignOperator(assignedVar, new MutableObject<>(f));
         a.getInputs().add(topOp);
+        a.setSourceLocation(sourceLoc);
 
         return new Pair<>(a, assignedVar);
 
@@ -1410,18 +1576,26 @@
     }
 
     protected ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        SourceLocation sourceLoc = expr.getSourceLocation();
         List<Mutable<ILogicalExpression>> argRefs = new ArrayList<>();
         argRefs.add(new MutableObject<>(expr));
         switch (expr.getExpressionTag()) {
             case CONSTANT:
             case VARIABLE:
-                return new UnnestingFunctionCallExpression(
+                UnnestingFunctionCallExpression scanCollExpr1 = new UnnestingFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                scanCollExpr1.setSourceLocation(sourceLoc);
+                return scanCollExpr1;
             case FUNCTION_CALL:
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                return (fce.getKind() == FunctionKind.UNNEST) ? expr
-                        : new UnnestingFunctionCallExpression(
-                                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    UnnestingFunctionCallExpression scanCollExpr2 = new UnnestingFunctionCallExpression(
+                            FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                    scanCollExpr2.setSourceLocation(sourceLoc);
+                    return scanCollExpr2;
+                }
             default:
                 return expr;
         }
@@ -1564,22 +1738,31 @@
     protected Pair<ILogicalOperator, LogicalVariable> constructSubplanOperatorForBranch(ILogicalOperator inputOp,
             Mutable<ILogicalExpression> selectExpr, Expression branchExpression) throws CompilationException {
         context.enterSubplan();
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         SubplanOperator subplanOp = new SubplanOperator();
         subplanOp.getInputs().add(new MutableObject<>(inputOp));
-        Mutable<ILogicalOperator> nestedSource =
-                new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
+        subplanOp.setSourceLocation(sourceLoc);
+        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+        ntsOp.setSourceLocation(sourceLoc);
         SelectOperator select = new SelectOperator(selectExpr, false, null);
         // The select operator cannot be moved up and down, otherwise it will cause
         // typing issues (ASTERIXDB-1203).
         OperatorPropertiesUtil.markMovable(select, false);
-        select.getInputs().add(nestedSource);
+        select.getInputs().add(new MutableObject<>(ntsOp));
+        select.setSourceLocation(selectExpr.getValue().getSourceLocation());
+
         Pair<ILogicalOperator, LogicalVariable> pBranch = branchExpression.accept(this, new MutableObject<>(select));
         LogicalVariable branchVar = context.newVar();
+        VariableReferenceExpression pBranchVarRef = new VariableReferenceExpression(pBranch.second);
+        pBranchVarRef.setSourceLocation(branchExpression.getSourceLocation());
+        AggregateFunctionCallExpression listifyExpr =
+                new AggregateFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false,
+                        Collections.singletonList(new MutableObject<>(pBranchVarRef)));
+        listifyExpr.setSourceLocation(branchExpression.getSourceLocation());
         AggregateOperator aggOp = new AggregateOperator(Collections.singletonList(branchVar),
-                Collections.singletonList(new MutableObject<>(new AggregateFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false, Collections.singletonList(
-                                new MutableObject<>(new VariableReferenceExpression(pBranch.second)))))));
+                Collections.singletonList(new MutableObject<>(listifyExpr)));
         aggOp.getInputs().add(new MutableObject<>(pBranch.first));
+        aggOp.setSourceLocation(branchExpression.getSourceLocation());
         ILogicalPlan planForBranch = new ALogicalPlanImpl(new MutableObject<>(aggOp));
         subplanOp.getNestedPlans().add(planForBranch);
         context.exitSubplan();
@@ -1587,22 +1770,27 @@
     }
 
     // Processes EXISTS and NOT EXISTS.
-    private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not) {
+    private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not,
+            SourceLocation sourceLoc) {
         AbstractFunctionCallExpression count =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCALAR_COUNT));
         count.getArguments().add(new MutableObject<>(inputExpr));
+        count.setSourceLocation(sourceLoc);
         AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(not ? BuiltinFunctions.EQ : BuiltinFunctions.NEQ));
         comparison.getArguments().add(new MutableObject<>(count));
         comparison.getArguments()
                 .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0L)))));
-        return new AssignOperator(v1, new MutableObject<>(comparison));
+        comparison.setSourceLocation(sourceLoc);
+        AssignOperator a = new AssignOperator(v1, new MutableObject<>(comparison));
+        a.setSourceLocation(sourceLoc);
+        return a;
     }
 
     // Generates the filter condition for whether a conditional branch should be
     // executed.
     protected Mutable<ILogicalExpression> generateNoMatchedPrecedingWhenBranchesFilter(
-            List<ILogicalExpression> inputBooleanExprs) {
+            List<ILogicalExpression> inputBooleanExprs, SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
         for (ILogicalExpression inputBooleanExpr : inputBooleanExprs) {
             // A NULL/MISSING valued WHEN expression does not lead to the corresponding THEN
@@ -1610,44 +1798,60 @@
             // Therefore, we should check a previous WHEN boolean condition is not unknown.
             arguments.add(generateAndNotIsUnknownWrap(inputBooleanExpr));
         }
-        Mutable<ILogicalExpression> hasBeenExecutedExprRef = new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.OR), arguments));
-        return new MutableObject<>(new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
-                new ArrayList<>(Collections.singletonList(hasBeenExecutedExprRef))));
+        ScalarFunctionCallExpression hasBeenExecutedExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.OR), arguments);
+        hasBeenExecutedExpr.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression notExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(hasBeenExecutedExpr))));
+        notExpr.setSourceLocation(sourceLoc);
+        return new MutableObject<>(notExpr);
     }
 
     // For an input expression `expr`, return `expr AND expr IS NOT UNKOWN`.
     protected Mutable<ILogicalExpression> generateAndNotIsUnknownWrap(ILogicalExpression logicalExpr) {
+        SourceLocation sourceLoc = logicalExpr.getSourceLocation();
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
         arguments.add(new MutableObject<>(logicalExpr));
-        Mutable<ILogicalExpression> expr = new MutableObject<>(
+        ScalarFunctionCallExpression isUnknownExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_UNKNOWN),
-                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr)))));
-        arguments.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), new ArrayList<>(Collections.singletonList(expr)))));
-        return new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), arguments));
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr))));
+        isUnknownExpr.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression notExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(isUnknownExpr))));
+        notExpr.setSourceLocation(sourceLoc);
+        arguments.add(new MutableObject<>(notExpr));
+        ScalarFunctionCallExpression andExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), arguments);
+        andExpr.setSourceLocation(sourceLoc);
+        return new MutableObject<>(andExpr);
     }
 
     // Generates the plan for "UNION ALL" or union expression from its input
     // expressions.
     protected Pair<ILogicalOperator, LogicalVariable> translateUnionAllFromInputExprs(List<ILangExpression> inputExprs,
-            Mutable<ILogicalOperator> tupSource) throws CompilationException {
+            Mutable<ILogicalOperator> tupSource, SourceLocation sourceLoc) throws CompilationException {
         List<Mutable<ILogicalOperator>> inputOpRefsToUnion = new ArrayList<>();
         List<LogicalVariable> vars = new ArrayList<>();
         for (ILangExpression expr : inputExprs) {
+            SourceLocation exprSourceLoc = expr.getSourceLocation();
             // Visits the expression of one branch.
             Pair<ILogicalOperator, LogicalVariable> opAndVar = expr.accept(this, tupSource);
 
             // Creates an unnest operator.
             LogicalVariable unnestVar = context.newVar();
             List<Mutable<ILogicalExpression>> args = new ArrayList<>();
-            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(opAndVar.second)));
-            UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                    new MutableObject<ILogicalExpression>(new UnnestingFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), args)));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(opAndVar.second);
+            varRef.setSourceLocation(exprSourceLoc);
+            args.add(new MutableObject<>(varRef));
+            UnnestingFunctionCallExpression scanCollExpr = new UnnestingFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), args);
+            scanCollExpr.setSourceLocation(exprSourceLoc);
+            UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollExpr));
             unnestOp.getInputs().add(new MutableObject<>(opAndVar.first));
-            inputOpRefsToUnion.add(new MutableObject<ILogicalOperator>(unnestOp));
+            unnestOp.setSourceLocation(exprSourceLoc);
+            inputOpRefsToUnion.add(new MutableObject<>(unnestOp));
             vars.add(unnestVar);
         }
 
@@ -1671,6 +1875,7 @@
             topUnionAllOp = new UnionAllOperator(varTriples);
             topUnionAllOp.getInputs().add(leftInputBranch);
             topUnionAllOp.getInputs().add(inputOpRefIterator.next());
+            topUnionAllOp.setSourceLocation(sourceLoc);
 
             // Re-assigns leftInputBranch and leftInputVar.
             leftInputBranch = new MutableObject<>(topUnionAllOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
index cb6d8e5..89619e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
@@ -63,7 +63,7 @@
                 if (fmtString != null) {
                     String format = ("JSON".equalsIgnoreCase(fmtString) || "CLEAN_JSON".equalsIgnoreCase(fmtString))
                             ? "JSON" : fmtString;
-                    return PlanFormat.valueOf(format);
+                    return PlanFormat.valueOf(format.toUpperCase());
                 }
             } catch (IllegalArgumentException e) {
                 logger.log(Level.INFO, fmtString + ": unsupported " + label + ", using " + defaultFmt + "instead", e);
@@ -129,7 +129,7 @@
 
     // Output format.
     private final OutputFormat fmt;
-    private final PlanFormat lpfmt;
+    private final PlanFormat planFormat;
 
     // Standard execution flags.
     private final boolean executeQuery;
@@ -143,8 +143,8 @@
         this(fmt, PlanFormat.STRING);
     }
 
-    public SessionConfig(OutputFormat fmt, PlanFormat lpfmt) {
-        this(fmt, true, true, true, lpfmt);
+    public SessionConfig(OutputFormat fmt, PlanFormat planFormat) {
+        this(fmt, true, true, true, planFormat);
     }
 
     /**
@@ -168,13 +168,13 @@
     }
 
     public SessionConfig(OutputFormat fmt, boolean optimize, boolean executeQuery, boolean generateJobSpec,
-            PlanFormat lpfmt) {
+            PlanFormat planFormat) {
         this.fmt = fmt;
         this.optimize = optimize;
         this.executeQuery = executeQuery;
         this.generateJobSpec = generateJobSpec;
         this.flags = new HashMap<>();
-        this.lpfmt = lpfmt;
+        this.planFormat = planFormat;
     }
 
     /**
@@ -187,8 +187,8 @@
     /**
      * Retrieve the PlanFormat for this execution.
      */
-    public PlanFormat getLpfmt() {
-        return this.lpfmt;
+    public PlanFormat getPlanFormat() {
+        return this.planFormat;
     }
 
     /**
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 9e36ba9..5c7b165 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -26,6 +26,7 @@
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Clause.ClauseType;
 import org.apache.asterix.lang.common.base.Expression;
@@ -70,7 +71,6 @@
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.commons.lang3.NotImplementedException;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -87,6 +87,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestNonMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
@@ -98,6 +99,7 @@
 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.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Each visit returns a pair of an operator and a variable. The variable
@@ -121,6 +123,7 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(Query q, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         Expression queryBody = q.getBody();
+        SourceLocation sourceLoc = queryBody.getSourceLocation();
         if (queryBody.getKind() == Kind.SELECT_EXPRESSION) {
             SelectExpression selectExpr = (SelectExpression) queryBody;
             if (q.isTopLevel()) {
@@ -132,8 +135,10 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(queryBody, tupSource);
             AssignOperator assignOp = new AssignOperator(var, new MutableObject<>(eo.first));
             assignOp.getInputs().add(eo.second);
+            assignOp.setSourceLocation(sourceLoc);
             ProjectOperator projectOp = new ProjectOperator(var);
             projectOp.getInputs().add(new MutableObject<>(assignOp));
+            projectOp.setSourceLocation(sourceLoc);
             return new Pair<>(projectOp, var);
         }
     }
@@ -175,21 +180,32 @@
             return leftInput.accept(this, tupSource);
         }
         List<ILangExpression> inputExprs = new ArrayList<>();
-        inputExprs.add(leftInput.selectBlock()
-                ? new SelectExpression(null, new SelectSetOperation(leftInput, null), null, null, true)
-                : leftInput.getSubquery());
+        SelectExpression leftInputExpr;
+        if (leftInput.selectBlock()) {
+            leftInputExpr = new SelectExpression(null, new SelectSetOperation(leftInput, null), null, null, true);
+            leftInputExpr.setSourceLocation(leftInput.getSelectBlock().getSourceLocation());
+        } else {
+            leftInputExpr = leftInput.getSubquery();
+        }
+        inputExprs.add(leftInputExpr);
         for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
             SetOpType setOpType = setOperationRight.getSetOpType();
             if (setOpType != SetOpType.UNION || setOperationRight.isSetSemantics()) {
-                throw new CompilationException("Operation " + setOpType
-                        + (setOperationRight.isSetSemantics() ? " with set semantics" : "ALL") + " is not supported.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, selectSetOperation.getSourceLocation(),
+                        "Operation " + setOpType + (setOperationRight.isSetSemantics() ? " with set semantics" : "ALL")
+                                + " is not supported.");
             }
             SetOperationInput rightInput = setOperationRight.getSetOperationRightInput();
-            inputExprs.add(rightInput.selectBlock()
-                    ? new SelectExpression(null, new SelectSetOperation(rightInput, null), null, null, true)
-                    : rightInput.getSubquery());
+            SelectExpression rightInputExpr;
+            if (rightInput.selectBlock()) {
+                rightInputExpr = new SelectExpression(null, new SelectSetOperation(rightInput, null), null, null, true);
+                rightInputExpr.setSourceLocation(rightInput.getSelectBlock().getSourceLocation());
+            } else {
+                rightInputExpr = rightInput.getSubquery();
+            }
+            inputExprs.add(rightInputExpr);
         }
-        return translateUnionAllFromInputExprs(inputExprs, tupSource);
+        return translateUnionAllFromInputExprs(inputExprs, tupSource, selectSetOperation.getSourceLocation());
     }
 
     @Override
@@ -236,10 +252,11 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(FromTerm fromTerm, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = fromTerm.getSourceLocation();
         LogicalVariable fromVar = context.newVarFromExpression(fromTerm.getLeftVariable());
         Expression fromExpr = fromTerm.getLeftExpression();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(fromExpr, tupSource);
-        ILogicalOperator unnestOp;
+        UnnestOperator unnestOp;
         if (fromTerm.hasPositionalVariable()) {
             LogicalVariable pVar = context.newVarFromExpression(fromTerm.getPositionalVariable());
             // We set the positional variable type as BIGINT type.
@@ -249,6 +266,7 @@
             unnestOp = new UnnestOperator(fromVar, new MutableObject<>(makeUnnestExpression(eo.first)));
         }
         unnestOp.getInputs().add(eo.second);
+        unnestOp.setSourceLocation(sourceLoc);
 
         // Processes joins, unnests, and nests.
         Mutable<ILogicalOperator> topOpRef = new MutableObject<>(unnestOp);
@@ -270,6 +288,7 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(JoinClause joinClause, Mutable<ILogicalOperator> inputRef)
             throws CompilationException {
+        SourceLocation sourceLoc = joinClause.getSourceLocation();
         Mutable<ILogicalOperator> leftInputRef = uncorrelatedLeftBranchStack.pop();
         if (joinClause.getJoinType() == JoinType.INNER) {
             Pair<ILogicalOperator, LogicalVariable> rightBranch =
@@ -277,6 +296,7 @@
             // A join operator with condition TRUE.
             AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
                     new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
+            joinOperator.setSourceLocation(sourceLoc);
             Mutable<ILogicalOperator> joinOpRef = new MutableObject<>(joinOperator);
 
             // Add an additional filter operator.
@@ -284,13 +304,16 @@
                     langExprToAlgExpression(joinClause.getConditionExpression(), joinOpRef);
             SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
             filter.getInputs().add(conditionExprOpPair.second);
+            filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
             return new Pair<>(filter, rightBranch.second);
         } else {
             // Creates a subplan operator.
             SubplanOperator subplanOp = new SubplanOperator();
-            Mutable<ILogicalOperator> ntsRef =
-                    new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
             subplanOp.getInputs().add(leftInputRef);
+            subplanOp.setSourceLocation(sourceLoc);
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            Mutable<ILogicalOperator> ntsRef = new MutableObject<>(ntsOp);
 
             // Enters the translation for a subplan.
             context.enterSubplan();
@@ -305,27 +328,36 @@
                     langExprToAlgExpression(joinClause.getConditionExpression(), new MutableObject<>(rightUnnestOp));
             SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
             filter.getInputs().add(conditionExprOpPair.second);
+            filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
 
             ILogicalOperator currentTopOp = filter;
             LogicalVariable varToListify;
             boolean hasRightPosVar = rightUnnestOp.getPositionalVariable() != null;
             if (hasRightPosVar) {
                 // Creates record to get correlation between the two aggregate variables.
+                VariableReferenceExpression rightUnnestVarRef =
+                        new VariableReferenceExpression(rightUnnestOp.getVariable());
+                rightUnnestVarRef.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
+                VariableReferenceExpression rightUnnestPosVarRef =
+                        new VariableReferenceExpression(rightUnnestOp.getPositionalVariable());
+                rightUnnestPosVarRef.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
                 ScalarFunctionCallExpression recordCreationFunc = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR),
                         // Field name for the listified right unnest var.
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AString("unnestvar")))),
                         // The listified right unnest var
-                        new MutableObject<>(new VariableReferenceExpression(rightUnnestOp.getVariable())),
+                        new MutableObject<>(rightUnnestVarRef),
                         // Field name for the listified right unnest positional var.
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AString("posvar")))),
                         // The listified right unnest positional var.
-                        new MutableObject<>(new VariableReferenceExpression(rightUnnestOp.getPositionalVariable())));
+                        new MutableObject<>(rightUnnestPosVarRef));
+                recordCreationFunc.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
 
                 // Assigns the record constructor function to a record variable.
                 LogicalVariable recordVar = context.newVar();
                 AssignOperator assignOp = new AssignOperator(recordVar, new MutableObject<>(recordCreationFunc));
                 assignOp.getInputs().add(new MutableObject<>(currentTopOp));
+                assignOp.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
 
                 // Sets currentTopOp and varToListify for later usages.
                 currentTopOp = assignOp;
@@ -335,14 +367,17 @@
             }
 
             // Adds an aggregate operator to listfy unnest variables.
-            AggregateFunctionCallExpression fListify =
-                    BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY,
-                            mkSingletonArrayList(new MutableObject<>(new VariableReferenceExpression(varToListify))));
+            VariableReferenceExpression varToListifyRef = new VariableReferenceExpression(varToListify);
+            varToListifyRef.setSourceLocation(currentTopOp.getSourceLocation());
+            AggregateFunctionCallExpression fListify = BuiltinFunctions.makeAggregateFunctionExpression(
+                    BuiltinFunctions.LISTIFY, mkSingletonArrayList(new MutableObject<>(varToListifyRef)));
+            fListify.setSourceLocation(currentTopOp.getSourceLocation());
 
             LogicalVariable aggVar = context.newSubplanOutputVar();
             AggregateOperator aggOp = new AggregateOperator(mkSingletonArrayList(aggVar),
                     mkSingletonArrayList(new MutableObject<>(fListify)));
             aggOp.getInputs().add(new MutableObject<>(currentTopOp));
+            aggOp.setSourceLocation(fListify.getSourceLocation());
 
             // Exits the translation of a subplan.
             context.exitSubplan();
@@ -353,20 +388,30 @@
 
             // Outer unnest the aggregated var from the subplan.
             LogicalVariable outerUnnestVar = context.newVar();
+            VariableReferenceExpression aggVarRefExpr = new VariableReferenceExpression(aggVar);
+            aggVarRefExpr.setSourceLocation(aggOp.getSourceLocation());
             LeftOuterUnnestOperator outerUnnestOp = new LeftOuterUnnestOperator(outerUnnestVar,
-                    new MutableObject<>(makeUnnestExpression(new VariableReferenceExpression(aggVar))));
+                    new MutableObject<>(makeUnnestExpression(aggVarRefExpr)));
             outerUnnestOp.getInputs().add(new MutableObject<>(subplanOp));
+            outerUnnestOp.setSourceLocation(aggOp.getSourceLocation());
             currentTopOp = outerUnnestOp;
 
             if (hasRightPosVar) {
+                VariableReferenceExpression outerUnnestVarRef1 = new VariableReferenceExpression(outerUnnestVar);
+                outerUnnestVarRef1.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
                 ScalarFunctionCallExpression fieldAccessForRightUnnestVar = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<>(new VariableReferenceExpression(outerUnnestVar)),
+                        new MutableObject<>(outerUnnestVarRef1),
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
+                fieldAccessForRightUnnestVar.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
+
+                VariableReferenceExpression outerUnnestVarRef2 = new VariableReferenceExpression(outerUnnestVar);
+                outerUnnestVarRef2.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
                 ScalarFunctionCallExpression fieldAccessForRightPosVar = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<>(new VariableReferenceExpression(outerUnnestVar)),
+                        new MutableObject<>(outerUnnestVarRef2),
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
+                fieldAccessForRightPosVar.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
 
                 // Creates variables for assign.
                 LogicalVariable rightUnnestVar = context.newVar();
@@ -389,6 +434,7 @@
                 // Creates the assign operator.
                 AssignOperator assignOp = new AssignOperator(assignVars, assignExprs);
                 assignOp.getInputs().add(new MutableObject<>(currentTopOp));
+                assignOp.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
                 currentTopOp = assignOp;
             } else {
                 context.setVar(joinClause.getRightVariable(), outerUnnestVar);
@@ -400,7 +446,8 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(NestClause nestClause, Mutable<ILogicalOperator> arg)
             throws CompilationException {
-        throw new NotImplementedException("Nest clause has not been implemented");
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, nestClause.getSourceLocation(),
+                "Nest clause has not been implemented");
     }
 
     @Override
@@ -426,7 +473,7 @@
         LogicalVariable rightVar = context.newVarFromExpression(binaryCorrelate.getRightVariable());
         Expression rightExpr = binaryCorrelate.getRightExpression();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(rightExpr, inputOpRef);
-        ILogicalOperator unnestOp;
+        AbstractUnnestOperator unnestOp;
         if (binaryCorrelate.hasPositionalVariable()) {
             LogicalVariable pVar = context.newVarFromExpression(binaryCorrelate.getPositionalVariable());
             // We set the positional variable type as BIGINT type.
@@ -440,6 +487,7 @@
                     : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(makeUnnestExpression(eo.first)));
         }
         unnestOp.getInputs().add(eo.second);
+        unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
         return new Pair<>(unnestOp, rightVar);
     }
 
@@ -478,12 +526,14 @@
         List<ILogicalExpression> branchCondVarReferences = new ArrayList<>();
         List<ILogicalExpression> allVarReferences = new ArrayList<>();
         for (int index = 0; index < whenExprList.size(); ++index) {
-            Pair<ILogicalOperator, LogicalVariable> whenExprResult = whenExprList.get(index).accept(this, currentOpRef);
+            Expression whenExpr = whenExprList.get(index);
+            Pair<ILogicalOperator, LogicalVariable> whenExprResult = whenExpr.accept(this, currentOpRef);
             currentOperator = whenExprResult.first;
             // Variable whenConditionVar is corresponds to the current "WHEN" condition.
             LogicalVariable whenConditionVar = whenExprResult.second;
-            Mutable<ILogicalExpression> branchEntraceConditionExprRef =
-                    new MutableObject<>(new VariableReferenceExpression(whenConditionVar));
+            VariableReferenceExpression whenConditionVarRef1 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef1.setSourceLocation(whenExpr.getSourceLocation());
+            Mutable<ILogicalExpression> branchEntraceConditionExprRef = new MutableObject<>(whenConditionVarRef1);
 
             // Constructs an expression that filters data based on preceding "WHEN" conditions
             // and the current "WHEN" condition. Note that only one "THEN" expression can be run
@@ -492,31 +542,45 @@
                 // The additional filter generated here makes sure the the tuple has not
                 // entered other matched "WHEN...THEN" case.
                 List<Mutable<ILogicalExpression>> andArgs = new ArrayList<>();
-                andArgs.add(generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences));
+                andArgs.add(generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences,
+                        caseExpression.getSourceLocation()));
                 andArgs.add(branchEntraceConditionExprRef);
 
                 // A "THEN" branch can be entered only when the tuple has not enter any other preceding
                 // branches and the current "WHEN" condition is TRUE.
-                branchEntraceConditionExprRef = new MutableObject<>(
-                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), andArgs));
+                ScalarFunctionCallExpression andExpr =
+                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), andArgs);
+                andExpr.setSourceLocation(whenExpr.getSourceLocation());
+                branchEntraceConditionExprRef = new MutableObject<>(andExpr);
             }
 
             // Translates the corresponding "THEN" expression.
-            Pair<ILogicalOperator, LogicalVariable> opAndVarForThen = constructSubplanOperatorForBranch(currentOperator,
-                    branchEntraceConditionExprRef, thenExprList.get(index));
+            Expression thenExpr = thenExprList.get(index);
+            Pair<ILogicalOperator, LogicalVariable> opAndVarForThen =
+                    constructSubplanOperatorForBranch(currentOperator, branchEntraceConditionExprRef, thenExpr);
 
-            branchCondVarReferences.add(new VariableReferenceExpression(whenConditionVar));
-            allVarReferences.add(new VariableReferenceExpression(whenConditionVar));
-            allVarReferences.add(new VariableReferenceExpression(opAndVarForThen.second));
+            VariableReferenceExpression whenConditionVarRef2 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef2.setSourceLocation(whenExpr.getSourceLocation());
+            branchCondVarReferences.add(whenConditionVarRef2);
+
+            VariableReferenceExpression whenConditionVarRef3 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef3.setSourceLocation(whenExpr.getSourceLocation());
+            allVarReferences.add(whenConditionVarRef3);
+
+            VariableReferenceExpression thenVarRef = new VariableReferenceExpression(opAndVarForThen.second);
+            thenVarRef.setSourceLocation(thenExpr.getSourceLocation());
+            allVarReferences.add(thenVarRef);
+
             currentOperator = opAndVarForThen.first;
             currentOpRef = new MutableObject<>(currentOperator);
         }
 
         // Creates a subplan for the "ELSE" branch.
-        Mutable<ILogicalExpression> elseCondExprRef =
-                generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences);
+        Mutable<ILogicalExpression> elseCondExprRef = generateNoMatchedPrecedingWhenBranchesFilter(
+                branchCondVarReferences, caseExpression.getSourceLocation());
+        Expression elseExpr = caseExpression.getElseExpr();
         Pair<ILogicalOperator, LogicalVariable> opAndVarForElse =
-                constructSubplanOperatorForBranch(currentOperator, elseCondExprRef, caseExpression.getElseExpr());
+                constructSubplanOperatorForBranch(currentOperator, elseCondExprRef, elseExpr);
 
         // Uses switch-case function to select the results of two branches.
         LogicalVariable selectVar = context.newVar();
@@ -525,25 +589,35 @@
         for (ILogicalExpression argVar : allVarReferences) {
             arguments.add(new MutableObject<>(argVar));
         }
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForElse.second)));
+        VariableReferenceExpression varForElseRef = new VariableReferenceExpression(opAndVarForElse.second);
+        varForElseRef.setSourceLocation(elseExpr.getSourceLocation());
+        arguments.add(new MutableObject<>(varForElseRef));
         AbstractFunctionCallExpression swithCaseExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SWITCH_CASE), arguments);
+        swithCaseExpr.setSourceLocation(caseExpression.getSourceLocation());
         AssignOperator assignOp = new AssignOperator(selectVar, new MutableObject<>(swithCaseExpr));
         assignOp.getInputs().add(new MutableObject<>(opAndVarForElse.first));
+        assignOp.setSourceLocation(caseExpression.getSourceLocation());
 
         // Unnests the selected (a "THEN" or "ELSE" branch) result.
         LogicalVariable unnestVar = context.newVar();
-        UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                new MutableObject<>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), Collections
-                                .singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
+        VariableReferenceExpression selectVarRef = new VariableReferenceExpression(selectVar);
+        selectVarRef.setSourceLocation(caseExpression.getSourceLocation());
+        UnnestingFunctionCallExpression scanCollectionExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        Collections.singletonList(new MutableObject<>(selectVarRef)));
+        scanCollectionExpr.setSourceLocation(caseExpression.getSourceLocation());
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollectionExpr));
         unnestOp.getInputs().add(new MutableObject<>(assignOp));
+        unnestOp.setSourceLocation(caseExpression.getSourceLocation());
 
         // Produces the final assign operator.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator finalAssignOp =
-                new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(caseExpression.getSourceLocation());
+        AssignOperator finalAssignOp = new AssignOperator(resultVar, new MutableObject<>(unnestVarRef));
         finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
+        finalAssignOp.setSourceLocation(caseExpression.getSourceLocation());
         return new Pair<>(finalAssignOp, resultVar);
     }
 
@@ -554,6 +628,7 @@
         } else {
             ProjectOperator pr = new ProjectOperator(resVar);
             pr.getInputs().add(returnOpRef);
+            pr.setSourceLocation(returnOpRef.getValue().getSourceLocation());
             return new Pair<>(pr, resVar);
         }
     }
@@ -567,19 +642,25 @@
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(returnExpr, tupSrc);
         LogicalVariable returnVar;
         ILogicalOperator returnOperator;
+        SourceLocation sourceLoc = returnExpr.getSourceLocation();
         if (returnExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
             VariableExpr varExpr = (VariableExpr) returnExpr;
             returnOperator = eo.second.getValue();
             returnVar = context.getVar(varExpr.getVar().getId());
         } else {
             returnVar = context.newVar();
-            returnOperator = new AssignOperator(returnVar, new MutableObject<>(eo.first));
-            returnOperator.getInputs().add(eo.second);
+            AssignOperator assignOp = new AssignOperator(returnVar, new MutableObject<>(eo.first));
+            assignOp.getInputs().add(eo.second);
+            assignOp.setSourceLocation(sourceLoc);
+            returnOperator = assignOp;
         }
         if (selectClause.distinct()) {
-            DistinctOperator distinctOperator = new DistinctOperator(
-                    mkSingletonArrayList(new MutableObject<>(new VariableReferenceExpression(returnVar))));
+            VariableReferenceExpression returnVarRef = new VariableReferenceExpression(returnVar);
+            returnVarRef.setSourceLocation(sourceLoc);
+            DistinctOperator distinctOperator =
+                    new DistinctOperator(mkSingletonArrayList(new MutableObject<>(returnVarRef)));
             distinctOperator.getInputs().add(new MutableObject<>(returnOperator));
+            distinctOperator.setSourceLocation(returnOperator.getSourceLocation());
             return new Pair<>(distinctOperator, returnVar);
         } else {
             return new Pair<>(returnOperator, returnVar);
@@ -593,7 +674,9 @@
         for (Projection projection : selectRegular.getProjections()) {
             if (projection.varStar()) {
                 if (!fieldBindings.isEmpty()) {
-                    recordExprs.add(new RecordConstructor(new ArrayList<>(fieldBindings)));
+                    RecordConstructor recordConstr = new RecordConstructor(new ArrayList<>(fieldBindings));
+                    recordConstr.setSourceLocation(projection.getSourceLocation());
+                    recordExprs.add(recordConstr);
                     fieldBindings.clear();
                 }
                 recordExprs.add(projection.getExpression());
@@ -617,11 +700,19 @@
             }
         }
         if (!fieldBindings.isEmpty()) {
-            recordExprs.add(new RecordConstructor(fieldBindings));
+            RecordConstructor recordConstr = new RecordConstructor(fieldBindings);
+            recordConstr.setSourceLocation(selectRegular.getSourceLocation());
+            recordExprs.add(recordConstr);
         }
 
-        return recordExprs.size() == 1 ? recordExprs.get(0)
-                : new CallExpr(new FunctionSignature(BuiltinFunctions.RECORD_CONCAT_STRICT), recordExprs);
+        if (recordExprs.size() == 1) {
+            return recordExprs.get(0);
+        } else {
+            CallExpr recordConcatExpr =
+                    new CallExpr(new FunctionSignature(BuiltinFunctions.RECORD_CONCAT_STRICT), recordExprs);
+            recordConcatExpr.setSourceLocation(selectRegular.getSourceLocation());
+            return recordConcatExpr;
+        }
     }
 
     // Generates all field bindings according to the from clause.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
deleted file mode 100644
index 0660f23..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.translator;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-
-public class TranslationException extends CompilationException {
-    private static final long serialVersionUID = 685960054131778068L;
-
-    public TranslationException(String msg) {
-        super(msg);
-    }
-}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
index 919bdf2..d2466bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
@@ -36,43 +36,52 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class PlanTranslationUtil {
     private static final LogicalVariable DUMMY_VAR = new LogicalVariable(-1);
 
     public static void prepareMetaKeyAccessExpression(List<String> field, LogicalVariable resVar,
             List<Mutable<ILogicalExpression>> assignExpressions, List<LogicalVariable> vars,
-            List<Mutable<ILogicalExpression>> varRefs, IVariableContext context) {
+            List<Mutable<ILogicalExpression>> varRefs, IVariableContext context, SourceLocation sourceLoc) {
         IAObject value = (field.size() > 1) ? new AOrderedList(field) : new AString(field.get(0));
         ScalarFunctionCallExpression metaKeyFunction =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.META_KEY));
-        metaKeyFunction.getArguments()
-                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+        metaKeyFunction.setSourceLocation(sourceLoc);
+        VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+        resVarRef.setSourceLocation(sourceLoc);
+        metaKeyFunction.getArguments().add(new MutableObject<ILogicalExpression>(resVarRef));
         metaKeyFunction.getArguments()
                 .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(value))));
         assignExpressions.add(new MutableObject<ILogicalExpression>(metaKeyFunction));
         LogicalVariable v = context.newVar();
         vars.add(v);
         if (varRefs != null) {
-            varRefs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            varRefs.add(new MutableObject<ILogicalExpression>(vRef));
         }
     }
 
     public static void prepareVarAndExpression(List<String> field, LogicalVariable resVar, List<LogicalVariable> vars,
             List<Mutable<ILogicalExpression>> assignExpressions, List<Mutable<ILogicalExpression>> varRefs,
-            IVariableContext context) {
-        ScalarFunctionCallExpression f = createFieldAccessExpression(new VariableReferenceExpression(DUMMY_VAR), field);
+            IVariableContext context, SourceLocation sourceLoc) {
+        VariableReferenceExpression dummyVarRef = new VariableReferenceExpression(DUMMY_VAR);
+        dummyVarRef.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression f = createFieldAccessExpression(dummyVarRef, field, sourceLoc);
         f.substituteVar(DUMMY_VAR, resVar);
         assignExpressions.add(new MutableObject<ILogicalExpression>(f));
         LogicalVariable v = context.newVar();
         vars.add(v);
         if (varRefs != null) {
-            varRefs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            varRefs.add(new MutableObject<ILogicalExpression>(vRef));
         }
     }
 
     private static ScalarFunctionCallExpression createFieldAccessExpression(ILogicalExpression target,
-            List<String> field) {
+            List<String> field, SourceLocation sourceLoc) {
         FunctionIdentifier functionIdentifier;
         IAObject value;
         if (field.size() > 1) {
@@ -83,7 +92,9 @@
             value = new AString(field.get(0));
         }
         IFunctionInfo finfoAccess = FunctionUtil.getFunctionInfo(functionIdentifier);
-        return new ScalarFunctionCallExpression(finfoAccess, new MutableObject<>(target),
+        ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(finfoAccess, new MutableObject<>(target),
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(value))));
+        faExpr.setSourceLocation(sourceLoc);
+        return faExpr;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 4b4b2b0..ffb1dd5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * A util that can verify if a filter field, a list of partitioning expressions,
@@ -50,16 +51,17 @@
      *            the record type
      * @param filterField
      *            the full name of the field
+     * @param sourceLoc
      * @throws AlgebricksException
      *             if field is not found in record.
      *             if field type can't be a filter type.
      *             if field type is nullable.
      */
-    public static void validateFilterField(ARecordType recordType, List<String> filterField)
+    public static void validateFilterField(ARecordType recordType, List<String> filterField, SourceLocation sourceLoc)
             throws AlgebricksException {
         IAType fieldType = recordType.getSubFieldType(filterField);
         if (fieldType == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
+            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
                     RecordUtil.toFullyQualifiedName(filterField));
         }
         switch (fieldType.getTypeTag()) {
@@ -100,6 +102,7 @@
      *            the key sources (record vs. meta)
      * @param autogenerated
      *            true if auto generated, false otherwise
+     * @param sourceLoc
      * @return a list of partitioning expressions types
      * @throws AlgebricksException
      *             if composite key is autogenerated.
@@ -109,33 +112,42 @@
      *             if the field type can't be a primary key.
      */
     public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
-            List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated)
-            throws AlgebricksException {
+            List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated,
+            SourceLocation sourceLoc) throws AlgebricksException {
         List<IAType> partitioningExprTypes = new ArrayList<>(partitioningExprs.size());
         if (autogenerated) {
             if (partitioningExprs.size() > 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY);
+                throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY,
+                        sourceLoc);
             }
             List<String> fieldName = partitioningExprs.get(0);
             IAType fieldType = recType.getSubFieldType(fieldName);
             if (fieldType == null) {
                 String unTypeField = fieldName.get(0) == null ? "" : fieldName.get(0);
-                throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, unTypeField);
+                throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc, unTypeField);
             }
             partitioningExprTypes.add(fieldType);
             ATypeTag pkTypeTag = fieldType.getTypeTag();
             if (pkTypeTag != ATypeTag.UUID) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, pkTypeTag.name(),
-                        ATypeTag.UUID.name());
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, sourceLoc,
+                        pkTypeTag.name(), ATypeTag.UUID.name());
             }
         } else {
             partitioningExprTypes =
                     KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, partitioningExprs, keySourceIndicators);
-            for (int fidx = 0; fidx < partitioningExprTypes.size(); ++fidx) {
-                IAType fieldType = partitioningExprTypes.get(fidx);
+            for (int i = 0; i < partitioningExprs.size(); i++) {
+                List<String> partitioningExpr = partitioningExprs.get(i);
+                IAType fieldType = partitioningExprTypes.get(i);
                 if (fieldType == null) {
-                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
-                            RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
+                            RecordUtil.toFullyQualifiedName(partitioningExpr));
+                }
+                boolean nullable = KeyFieldTypeUtil.chooseSource(keySourceIndicators, i, recType, metaRecType)
+                        .isSubFieldNullable(partitioningExpr);
+                if (nullable) {
+                    // key field is nullable
+                    throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+                            RecordUtil.toFullyQualifiedName(partitioningExpr));
                 }
                 switch (fieldType.getTypeTag()) {
                     case TINYINT:
@@ -154,10 +166,10 @@
                     case DAYTIMEDURATION:
                         break;
                     case UNION:
-                        throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE,
-                                RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+                        throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+                                RecordUtil.toFullyQualifiedName(partitioningExpr));
                     default:
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE,
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE, sourceLoc,
                                 fieldType.getTypeTag());
                 }
             }
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index f06b6d2..c804153 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -153,6 +153,7 @@
           <usedDependencies combine.children="append">
             <usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
             <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
+            <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
           </usedDependencies>
           <ignoredUnusedDeclaredDependencies>
             <ignoredUnusedDeclaredDependency>org.apache.asterix:asterix-external-data:zip:*</ignoredUnusedDeclaredDependency>
@@ -643,5 +644,11 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-geo</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index ad715a4..a1d3f57 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -20,6 +20,7 @@
 
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -67,6 +68,7 @@
 import org.apache.asterix.optimizer.rules.am.AbstractIntroduceAccessMethodRule;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.ExecutionPlans;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
@@ -100,6 +102,7 @@
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.config.IOptionType;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.resource.IClusterCapacity;
@@ -120,8 +123,7 @@
     private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
     // one for query, two for intermediate results, one for final result, and one for reading an inverted list
     private static final int MIN_FRAME_LIMIT_FOR_TEXTSEARCH = 5;
-    private static final String LPLAN = "Logical plan";
-    private static final String OPLAN = "Optimized logical plan";
+    private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
 
     // A white list of supported configurable parameters.
     private static final Set<String> CONFIGURABLE_PARAMETER_NAMES =
@@ -137,12 +139,14 @@
     private final IAstPrintVisitorFactory astPrintVisitorFactory;
     private final ILangExpressionToPlanTranslatorFactory translatorFactory;
     private final IRuleSetFactory ruleSetFactory;
+    private final ExecutionPlans executionPlans;
 
     public APIFramework(ILangCompilationProvider compilationProvider) {
         this.rewriterFactory = compilationProvider.getRewriterFactory();
         this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
         this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
         this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+        executionPlans = new ExecutionPlans();
     }
 
     private static class OptimizationContextFactory implements IOptimizationContextFactory {
@@ -165,27 +169,6 @@
         }
     }
 
-    private void printPlanPrefix(SessionOutput output, String planName) {
-        if (output.config().is(SessionConfig.FORMAT_HTML)) {
-            output.out().println("<h4>" + planName + ":</h4>");
-            if (LPLAN.equalsIgnoreCase(planName)) {
-                output.out().println("<pre class = query-plan>");
-            } else if (OPLAN.equalsIgnoreCase(planName)) {
-                output.out().println("<pre class = query-optimized-plan>");
-            } else {
-                output.out().println("<pre>");
-            }
-        } else {
-            output.out().println("----------" + planName + ":");
-        }
-    }
-
-    private void printPlanPostfix(SessionOutput output) {
-        if (output.config().is(SessionConfig.FORMAT_HTML)) {
-            output.out().println("</pre>");
-        }
-    }
-
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
             MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs)
             throws CompilationException {
@@ -194,10 +177,7 @@
         }
         SessionConfig conf = output.config();
         if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_EXPR_TREE)) {
-            output.out().println();
-            printPlanPrefix(output, "Expression tree");
-            q.accept(astPrintVisitorFactory.createLangVisitor(output.out()), 0);
-            printPlanPostfix(output);
+            generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
         rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()), inlineUdfs);
@@ -211,16 +191,13 @@
         // establish facts
         final boolean isQuery = query != null;
         final boolean isLoad = statement != null && statement.getKind() == Statement.Kind.LOAD;
+        final SourceLocation sourceLoc =
+                query != null ? query.getSourceLocation() : statement != null ? statement.getSourceLocation() : null;
 
         SessionConfig conf = output.config();
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
-            output.out().println();
-
-            printPlanPrefix(output, "Rewritten expression tree");
-            if (isQuery) {
-                query.accept(astPrintVisitorFactory.createLangVisitor(output.out()), 0);
-            }
-            printPlanPostfix(output);
+        if (isQuery && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+                && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
+            generateRewrittenExpressionTree(query);
         }
 
         final TxnId txnId = metadataProvider.getTxnIdFactory().create();
@@ -230,19 +207,14 @@
 
         ILogicalPlan plan = isLoad ? t.translateLoad(statement) : t.translate(query, outputDatasetName, statement);
 
-        if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
-            output.out().println();
-
-            printPlanPrefix(output, "Logical plan");
-            if (isQuery || isLoad) {
-                PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(output.config().getLpfmt(), output.out()), 0);
-            }
-            printPlanPostfix(output);
+        if ((isQuery || isLoad) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+                && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
+            generateLogicalPlan(plan, output.config().getPlanFormat());
         }
         CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
-        Map<String, String> querySpecificConfig = validateConfig(metadataProvider.getConfig());
+        Map<String, String> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
-                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig);
+                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig, sourceLoc);
 
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
@@ -273,12 +245,9 @@
                     AlgebricksAppendable buffer = new AlgebricksAppendable(output.out());
                     PlanPrettyPrinter.printPhysicalOps(plan, buffer, 0);
                 } else {
-                    printPlanPrefix(output, "Optimized logical plan");
                     if (isQuery || isLoad) {
-                        PlanPrettyPrinter.printPlan(plan,
-                                getPrettyPrintVisitor(output.config().getLpfmt(), output.out()), 0);
+                        generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
                     }
-                    printPlanPostfix(output);
                 }
             }
         }
@@ -327,26 +296,27 @@
                     ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
             spec.setRequiredClusterCapacity(jobRequiredCapacity);
         }
-
-        printJobSpec(query, spec, conf, output);
+        if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
+            generateJob(spec);
+        }
         return spec;
     }
 
     protected PhysicalOptimizationConfig getPhysicalOptimizationConfig(CompilerProperties compilerProperties,
-            Map<String, String> querySpecificConfig) throws AlgebricksException {
+            Map<String, String> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
         int frameSize = compilerProperties.getFrameSize();
         int sortFrameLimit = getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
                 querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
-                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT);
+                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT, sourceLoc);
         int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
                 querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
-                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY);
+                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
         int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
                 querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
-                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN);
+                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
         int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
                 querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
-                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH);
+                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
         final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.getPhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
         physOptConf.setMaxFramesExternalSort(sortFrameLimit);
@@ -373,23 +343,6 @@
         }
     }
 
-    protected void printJobSpec(Query rwQ, JobSpecification spec, SessionConfig conf, SessionOutput output)
-            throws AlgebricksException {
-        if (conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
-            printPlanPrefix(output, "Hyracks job");
-            if (rwQ != null) {
-                try {
-                    final ObjectWriter objectWriter = new ObjectMapper().writerWithDefaultPrettyPrinter();
-                    output.out().println(objectWriter.writeValueAsString(spec.toJSON()));
-                } catch (IOException e) {
-                    throw new AlgebricksException(e);
-                }
-                output.out().println(spec.getUserConstraints());
-            }
-            printPlanPostfix(output);
-        }
-    }
-
     private AbstractLogicalOperatorPrettyPrintVisitor getPrettyPrintVisitor(SessionConfig.PlanFormat planFormat,
             PrintWriter out) {
         return planFormat.equals(SessionConfig.PlanFormat.JSON) ? new LogicalOperatorPrettyPrintVisitorJson(out)
@@ -429,6 +382,10 @@
         }
     }
 
+    public ExecutionPlans getExecutionPlans() {
+        return executionPlans;
+    }
+
     // Chooses the location constraints, i.e., whether to use storage parallelism or use a user-sepcified number
     // of cores.
     private static AlgebricksAbsolutePartitionConstraint chooseLocations(IClusterInfoCollector clusterInfoCollector,
@@ -496,12 +453,17 @@
 
     // Gets the frame limit.
     private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
-            int frameSize, int minFrameLimit) throws AlgebricksException {
+            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
         IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
-        long memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        long memBudget;
+        try {
+            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+        }
         int frameLimit = (int) (memBudget / frameSize);
         if (frameLimit < minFrameLimit) {
-            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, parameterName,
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
                     frameSize * minFrameLimit);
         }
         // Sets the frame limit to the minimum frame limit if the caculated frame limit is too small.
@@ -515,15 +477,60 @@
     }
 
     // Validates if the query contains unsupported query parameters.
-    private static Map<String, String> validateConfig(Map<String, String> config) throws AlgebricksException {
+    private static Map<String, String> validateConfig(Map<String, String> config, SourceLocation sourceLoc)
+            throws AlgebricksException {
         for (String parameterName : config.keySet()) {
             if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)) {
-                throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, parameterName);
+                throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
+                        parameterName);
             }
         }
         return config;
     }
 
+    private void generateExpressionTree(IReturningStatement statement) throws CompilationException {
+        final StringWriter stringWriter = new StringWriter();
+        try (PrintWriter writer = new PrintWriter(stringWriter)) {
+            statement.accept(astPrintVisitorFactory.createLangVisitor(writer), 0);
+            executionPlans.setExpressionTree(stringWriter.toString());
+        }
+    }
+
+    private void generateRewrittenExpressionTree(IReturningStatement statement) throws CompilationException {
+        final StringWriter stringWriter = new StringWriter();
+        try (PrintWriter writer = new PrintWriter(stringWriter)) {
+            statement.accept(astPrintVisitorFactory.createLangVisitor(writer), 0);
+            executionPlans.setRewrittenExpressionTree(stringWriter.toString());
+        }
+    }
+
+    private void generateLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format) throws AlgebricksException {
+        final StringWriter stringWriter = new StringWriter();
+        try (PrintWriter writer = new PrintWriter(stringWriter)) {
+            PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(format, writer), 0);
+            executionPlans.setLogicalPlan(stringWriter.toString());
+        }
+    }
+
+    private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format)
+            throws AlgebricksException {
+        final StringWriter stringWriter = new StringWriter();
+        try (PrintWriter writer = new PrintWriter(stringWriter)) {
+            PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(format, writer), 0);
+            executionPlans.setOptimizedLogicalPlan(stringWriter.toString());
+        }
+    }
+
+    private void generateJob(JobSpecification spec) {
+        final StringWriter stringWriter = new StringWriter();
+        try (PrintWriter writer = new PrintWriter(stringWriter)) {
+            writer.println(OBJECT_WRITER.writeValueAsString(spec.toJSON()));
+            executionPlans.setJob(stringWriter.toString());
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
     public static AlgebricksAbsolutePartitionConstraint getJobLocations(JobSpecification spec,
             INodeJobTracker jobTracker, AlgebricksAbsolutePartitionConstraint clusterLocations) {
         final Set<String> jobParticipatingNodes = jobTracker.getJobParticipatingNodes(spec);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index b8c737d..bd096dd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -51,7 +51,8 @@
         RESULTS("results"),
         HANDLE("handle"),
         ERRORS("errors"),
-        METRICS("metrics");
+        METRICS("metrics"),
+        PLANS("plans");
 
         private final String str;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
deleted file mode 100644
index 4c1dbf7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class DdlApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES =
-            Statement.Category.QUERY | Statement.Category.UPDATE | Statement.Category.DDL;
-
-    public DdlApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("ddl");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-DDL statement %s to the DDL API.";
-    }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
deleted file mode 100644
index 0a461c7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class FullApiServlet extends RestApiServlet {
-
-    private static final String AQL_STMT_PARAM_NAME = "aql";
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
-            | Statement.Category.DDL | Statement.Category.PROCEDURE;
-
-    public FullApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        throw new IllegalStateException();
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter(AQL_STMT_PARAM_NAME);
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index da621d2..5ad451f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -23,6 +23,7 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
 import org.apache.asterix.app.message.CancelQueryRequest;
 import org.apache.asterix.app.message.CancelQueryResponse;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
@@ -54,7 +55,7 @@
     @Override
     protected void delete(IServletRequest request, IServletResponse response) {
         // gets the parameter client_context_id from the request.
-        String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
+        String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
         if (clientContextId == null) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index a420efc..1713ca5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -132,6 +132,7 @@
         } else {
             sessionOutput.out().append(responseMsg.getResult());
         }
+        printExecutionPlans(sessionOutput, responseMsg.getExecutionPlans());
     }
 
     private void cancelQuery(INCMessageBroker messageBroker, String nodeId, String clientContextID, Exception exception,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
deleted file mode 100644
index 3e692d3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class QueryApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
-
-    public QueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("query");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-query statement %s to the query API.";
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
index f8655ad..a8b3aef 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
 import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
@@ -38,7 +39,6 @@
  */
 public class QueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
-    protected static final String CLIENT_CONTEXT_ID = "client_context_id";
 
     public QueryCancellationServlet(ConcurrentMap<String, Object> ctx, String... paths) {
         super(ctx, paths);
@@ -46,7 +46,7 @@
 
     @Override
     protected void delete(IServletRequest request, IServletResponse response) throws IOException {
-        String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
+        String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
         if (clientContextId == null) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 56359e3..3d0858c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -45,6 +45,8 @@
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
@@ -73,6 +75,7 @@
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+
 import io.netty.handler.codec.http.HttpResponseStatus;
 
 public class QueryServiceServlet extends AbstractQueryApiServlet {
@@ -141,7 +144,13 @@
         MODE("mode"),
         TIMEOUT("timeout"),
         PLAN_FORMAT("plan-format"),
-        MAX_RESULT_READS("max-result-reads");
+        MAX_RESULT_READS("max-result-reads"),
+        EXPRESSION_TREE("expression-tree"),
+        REWRITTEN_EXPRESSION_TREE("rewritten-expression-tree"),
+        LOGICAL_PLAN("logical-plan"),
+        OPTIMIZED_LOGICAL_PLAN("optimized-logical-plan"),
+        JOB("job"),
+        SIGNATURE("signature");
 
         private final String str;
 
@@ -188,7 +197,7 @@
         }
     }
 
-    static class RequestParameters {
+    protected static class RequestParameters {
         String host;
         String path;
         String statement;
@@ -198,6 +207,13 @@
         String clientContextID;
         String mode;
         String maxResultReads;
+        String planFormat;
+        boolean expressionTree;
+        boolean rewrittenExpressionTree;
+        boolean logicalPlan;
+        boolean optimizedLogicalPlan;
+        boolean job;
+        boolean signature;
 
         @Override
         public String toString() {
@@ -213,6 +229,13 @@
                 on.put("format", format);
                 on.put("timeout", timeout);
                 on.put("maxResultReads", maxResultReads);
+                on.put("planFormat", planFormat);
+                on.put("expressionTree", expressionTree);
+                on.put("rewrittenExpressionTree", rewrittenExpressionTree);
+                on.put("logicalPlan", logicalPlan);
+                on.put("optimizedLogicalPlan", optimizedLogicalPlan);
+                on.put("job", job);
+                on.put("signature", signature);
                 return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
             } catch (JsonProcessingException e) { // NOSONAR
                 LOGGER.debug("unexpected exception marshalling {} instance to json", getClass(), e);
@@ -221,7 +244,7 @@
         }
     }
 
-    static final class RequestExecutionState {
+    protected static final class RequestExecutionState {
         private long execStart = -1;
         private long execEnd = -1;
         private ResultStatus resultStatus = ResultStatus.SUCCESS;
@@ -291,7 +314,7 @@
             if (format.equals(HttpUtil.ContentType.APPLICATION_ADM)) {
                 return SessionConfig.OutputFormat.ADM;
             }
-            if (format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)) {
+            if (isJsonFormat(format)) {
                 return Boolean.parseBoolean(getParameterValue(format, Attribute.LOSSLESS.str()))
                         ? SessionConfig.OutputFormat.LOSSLESS_JSON : SessionConfig.OutputFormat.CLEAN_JSON;
             }
@@ -307,9 +330,15 @@
         SessionOutput.ResultAppender appendStatus = ResultUtil.createResultStatusAppender();
 
         SessionConfig.OutputFormat format = getFormat(param.format);
-        //TODO:get the parameters from UI.Currently set to clean_json.
-        SessionConfig sessionConfig = new SessionConfig(format);
+        final SessionConfig.PlanFormat planFormat =
+                SessionConfig.PlanFormat.get(param.planFormat, param.planFormat, SessionConfig.PlanFormat.JSON, LOGGER);
+        SessionConfig sessionConfig = new SessionConfig(format, planFormat);
         sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
+        sessionConfig.set(SessionConfig.OOB_EXPR_TREE, param.expressionTree);
+        sessionConfig.set(SessionConfig.OOB_REWRITTEN_EXPR_TREE, param.rewrittenExpressionTree);
+        sessionConfig.set(SessionConfig.OOB_LOGICAL_PLAN, param.logicalPlan);
+        sessionConfig.set(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN, param.optimizedLogicalPlan);
+        sessionConfig.set(SessionConfig.OOB_HYRACKS_JOB, param.job);
         sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.pretty);
         sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
                 format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
@@ -324,8 +353,15 @@
         }
     }
 
-    private static void printSignature(PrintWriter pw) {
-        ResultUtil.printField(pw, ResultFields.SIGNATURE.str(), "*");
+    private static void printSignature(PrintWriter pw, RequestParameters param) {
+        if (param.signature) {
+            pw.print("\t\"");
+            pw.print(ResultFields.SIGNATURE.str());
+            pw.print("\": {\n");
+            pw.print("\t");
+            ResultUtil.printField(pw, "*", "*", false);
+            pw.print("\t},\n");
+        }
     }
 
     private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
@@ -391,6 +427,14 @@
                 param.clientContextID = getOptText(jsonRequest, Parameter.CLIENT_ID.str());
                 param.timeout = getOptText(jsonRequest, Parameter.TIMEOUT.str());
                 param.maxResultReads = getOptText(jsonRequest, Parameter.MAX_RESULT_READS.str());
+                param.planFormat = getOptText(jsonRequest, Parameter.PLAN_FORMAT.str());
+                param.expressionTree = getOptBoolean(jsonRequest, Parameter.EXPRESSION_TREE.str(), false);
+                param.rewrittenExpressionTree =
+                        getOptBoolean(jsonRequest, Parameter.REWRITTEN_EXPRESSION_TREE.str(), false);
+                param.logicalPlan = getOptBoolean(jsonRequest, Parameter.LOGICAL_PLAN.str(), false);
+                param.optimizedLogicalPlan = getOptBoolean(jsonRequest, Parameter.OPTIMIZED_LOGICAL_PLAN.str(), false);
+                param.job = getOptBoolean(jsonRequest, Parameter.JOB.str(), false);
+                param.signature = getOptBoolean(jsonRequest, Parameter.SIGNATURE.str(), true);
             } catch (JsonParseException | JsonMappingException e) {
                 // if the JSON parsing fails, the statement is empty and we get an empty statement error
                 GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, e.getMessage(), e);
@@ -406,6 +450,7 @@
             param.clientContextID = request.getParameter(Parameter.CLIENT_ID.str());
             param.timeout = request.getParameter(Parameter.TIMEOUT.str());
             param.maxResultReads = request.getParameter(Parameter.MAX_RESULT_READS.str());
+            param.planFormat = request.getParameter(Parameter.PLAN_FORMAT.str());
         }
         return param;
     }
@@ -473,7 +518,7 @@
         sessionOutput.out().print("{\n");
         printRequestId(sessionOutput.out());
         printClientContextID(sessionOutput.out(), param);
-        printSignature(sessionOutput.out());
+        printSignature(sessionOutput.out(), param);
         printType(sessionOutput.out(), sessionConfig);
         long errorCount = 1; // so far we just return 1 error
         try {
@@ -498,7 +543,7 @@
         } catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
             handleExecuteStatementException(e, execution, param);
             response.setStatus(execution.getHttpStatus());
-            ResultUtil.printError(sessionOutput.out(), e);
+            printError(sessionOutput.out(), e);
             ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
         } finally {
             // make sure that we stop buffering and return the result to the http response
@@ -533,6 +578,7 @@
                 getHyracksDataset(), resultProperties, stats, null, param.clientContextID, optionalParameters);
         translator.compileAndExecute(getHyracksClientConnection(), queryCtx, requestParameters);
         execution.end();
+        printExecutionPlans(sessionOutput, translator.getExecutionPlans());
     }
 
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState state, RequestParameters param) {
@@ -566,4 +612,30 @@
             state.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
         }
     }
+
+    protected void printError(PrintWriter sessionOut, Throwable throwable) {
+        ResultUtil.printError(sessionOut, throwable);
+    }
+
+    protected void printExecutionPlans(SessionOutput output, ExecutionPlans executionPlans) {
+        final PrintWriter pw = output.out();
+        pw.print("\t\"");
+        pw.print(ResultFields.PLANS.str());
+        pw.print("\":");
+        final SessionConfig.PlanFormat planFormat = output.config().getPlanFormat();
+        switch (planFormat) {
+            case JSON:
+            case STRING:
+                pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
+                break;
+            default:
+                throw new IllegalStateException("Unrecognized plan format: " + planFormat);
+        }
+        pw.print(",\n");
+    }
+
+    private static boolean isJsonFormat(String format) {
+        return format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)
+                || format.equalsIgnoreCase(HttpUtil.ContentType.JSON);
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
index 8dd4290..6b632a1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.api.http.server;
 
+import static org.apache.hyracks.util.NetworkUtil.toHostPort;
+
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetSocketAddress;
@@ -131,8 +133,7 @@
             ArrayNode replicasArray = OBJECT_MAPPER.createArrayNode();
             for (IPartitionReplica replica : replicas) {
                 final ObjectNode replicaJson = OBJECT_MAPPER.createObjectNode();
-                final InetSocketAddress location = replica.getIdentifier().getLocation();
-                replicaJson.put("location", location.getHostString() + ":" + location.getPort());
+                replicaJson.put("location", toHostPort(replica.getIdentifier().getLocation()));
                 replicaJson.put("status", replica.getStatus().toString());
                 replicasArray.add(replicaJson);
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
deleted file mode 100644
index 3650189..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class UpdateApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
-
-    public UpdateApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("statements");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-Update statement %s to the Update API.";
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index e8bcb10..1b7d5b9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
@@ -33,7 +32,6 @@
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.metadata.api.IActiveEntityController;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.utils.DatasetUtil;
@@ -209,19 +207,12 @@
     }
 
     @Override
-    public synchronized void recover() {
+    public void recover() {
         LOGGER.log(level, "Starting active recovery");
-        for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
+        for (IActiveEntityEventsListener listener : getEventListeners()) {
             synchronized (listener) {
                 LOGGER.log(level, "Entity " + listener.getEntityId() + " is " + listener.getStats());
-                if (listener.getState() == ActivityState.PERMANENTLY_FAILED
-                        && listener instanceof IActiveEntityController) {
-                    LOGGER.log(level, "Recovering");
-                    ((IActiveEntityController) listener).recover();
-                } else {
-                    LOGGER.log(level, "Only notifying");
-                    listener.notifyAll();
-                }
+                listener.notifyAll();
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index 2de8319..3024dc6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -21,7 +21,6 @@
 import java.util.concurrent.Callable;
 
 import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.IRetryPolicy;
 import org.apache.asterix.active.IRetryPolicyFactory;
 import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -34,6 +33,7 @@
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.IRetryPolicy;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -93,6 +93,7 @@
                 synchronized (listener) {
                     if (!cancelRecovery) {
                         listener.setState(ActivityState.PERMANENTLY_FAILED);
+                        listener.setRunning(metadataProvider, false);
                     }
                 }
             } else {
@@ -112,7 +113,7 @@
             return null;
         }
         LOGGER.log(level, "calling the policy");
-        while (policy.retry()) {
+        while (policy.retry(failure)) {
             synchronized (listener) {
                 if (cancelRecovery) {
                     return null;
@@ -170,7 +171,9 @@
                     return null;
                 }
                 if (listener.getState() == ActivityState.TEMPORARILY_FAILED) {
+                    LOGGER.warn("Recovery for {} permanently failed", listener.getEntityId());
                     listener.setState(ActivityState.PERMANENTLY_FAILED);
+                    listener.setRunning(metadataProvider, false);
                 }
                 listener.notifyAll();
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 1ba418a..7f632f0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -28,12 +28,15 @@
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.common.api.ExtensionId;
 import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.hyracks.bootstrap.GlobalRecoveryManager;
 import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.runtime.functions.FunctionCollection;
@@ -41,6 +44,8 @@
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.utils.ExtensionUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
@@ -53,6 +58,7 @@
     private final ILangCompilationProvider aqlCompilationProvider;
     private final ILangCompilationProvider sqlppCompilationProvider;
     private final IFunctionManager functionManager;
+    private final IGlobalRecoveryExtension globalRecoveryExtension;
     private transient IStatementExecutorFactory statementExecutorFactory;
 
     /**
@@ -71,6 +77,7 @@
         Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
         Pair<ExtensionId, IFunctionManager> fm = null;
         IStatementExecutorExtension see = null;
+        IGlobalRecoveryExtension gre = null;
         if (list != null) {
             Set<ExtensionId> extensionIds = new HashSet<>();
             for (AsterixExtension extensionConf : list) {
@@ -89,6 +96,9 @@
                         sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
                         fm = ExtensionUtil.extendFunctionManager(fm, le);
                         break;
+                    case RECOVERY:
+                        gre = (IGlobalRecoveryExtension) extension;
+                        break;
                     default:
                         break;
                 }
@@ -99,6 +109,7 @@
         this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
         this.functionManager =
                 fm == null ? new FunctionManager(FunctionCollection.createDefaultFunctionCollection()) : fm.second;
+        this.globalRecoveryExtension = gre;
     }
 
     /** @deprecated use getStatementExecutorFactory instead */
@@ -127,6 +138,14 @@
         }
     }
 
+    public IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+            IStorageComponentProvider componentProvider) {
+        if (globalRecoveryExtension == null) {
+            return new GlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+        }
+        return globalRecoveryExtension.getGlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+    }
+
     @Override
     public IFunctionManager getFunctionManager() {
         return functionManager;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java
new file mode 100644
index 0000000..e9652a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.cc;
+
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+
+/**
+ * An interface for extensions of {@code IGlobalRecoveryManager}
+ */
+public interface IGlobalRecoveryExtension extends IExtension {
+
+    @Override
+    default ExtensionKind getExtensionKind() {
+        return ExtensionKind.RECOVERY;
+    }
+
+    IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+            IStorageComponentProvider componentProvider);
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
index a575ba4..47c1045 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.function;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -45,7 +47,8 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
         }
         return new DatasetResourcesDatasource(context.getComputationNodeDomain(), dataset.getDatasetId());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
index c857ce0..590bf54 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
@@ -22,6 +22,8 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -68,7 +70,8 @@
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
             // TODO remove this after enabling the support of positional variables in data scan
-            throw new AlgebricksException("No positional variables are allowed over datasets.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over datasets.");
         }
         ILogicalExpression expr = f.getArguments().get(0).getValue();
         if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
@@ -90,7 +93,8 @@
         String datasetName = datasetReference.second;
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, unnest.getSourceLocation(),
+                    datasetName, dataverseName);
         }
         DataSourceId asid = new DataSourceId(dataverseName, datasetName);
         List<LogicalVariable> variables = new ArrayList<>();
@@ -107,6 +111,7 @@
             variables.add(context.newVar());
         }
         DataSourceScanOperator scan = new DataSourceScanOperator(variables, dataSource);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
index 51aca5d..e9087c2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
@@ -21,6 +21,8 @@
 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.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.FeedUtils;
@@ -70,7 +72,8 @@
         AbstractFunctionCallExpression f = UnnestToDataScanRule.getFunctionCall(opRef);
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
-            throw new AlgebricksException("No positional variables are allowed over feeds.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over feeds.");
         }
         String dataverse = ConstantExpressionUtil.getStringArgument(f, 0);
         String sourceFeedName = ConstantExpressionUtil.getStringArgument(f, 1);
@@ -85,7 +88,8 @@
         if (policy == null) {
             policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
             if (policy == null) {
-                throw new AlgebricksException("Unknown feed policy:" + policyName);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                        "Unknown feed policy:" + policyName);
             }
         }
         ArrayList<LogicalVariable> feedDataScanOutputVariables = new ArrayList<>();
@@ -104,6 +108,7 @@
             feedDataScanOutputVariables.addAll(pkVars);
         }
         DataSourceScanOperator scan = new DataSourceScanOperator(feedDataScanOutputVariables, ds);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
@@ -152,10 +157,10 @@
                 List<String> key = partitioningKeys.get(i);
                 if (keySourceIndicator == null || keySourceIndicator.get(i).intValue() == 0) {
                     PlanTranslationUtil.prepareVarAndExpression(key, recordVar, pkVars, keyAccessExpression, null,
-                            context);
+                            context, null);
                 } else {
                     PlanTranslationUtil.prepareMetaKeyAccessExpression(key, recordVar, keyAccessExpression, pkVars,
-                            null, context);
+                            null, context, null);
                 }
             }
             keyAccessExpression.forEach(
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index 2ff9282..2bc5ab6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -21,6 +21,8 @@
 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.metadata.declared.FunctionDataSource;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -55,24 +57,29 @@
         AbstractFunctionCallExpression f = UnnestToDataScanRule.getFunctionCall(opRef);
         List<Mutable<ILogicalExpression>> args = f.getArguments();
         if (args.size() != functionId.getArity()) {
-            throw new AlgebricksException("Function " + functionId.getNamespace() + "." + functionId.getName()
-                    + " expects " + functionId.getArity() + " arguments");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, f.getSourceLocation(),
+                    "Function " + functionId.getNamespace() + "." + functionId.getName() + " expects "
+                            + functionId.getArity() + " arguments");
         }
         for (int i = 0; i < args.size(); i++) {
-            if (args.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                throw new AlgebricksException("Function " + functionId.getNamespace() + "." + functionId.getName()
-                        + " expects constant arguments while arg[" + i + "] is of type "
-                        + args.get(i).getValue().getExpressionTag());
+            ILogicalExpression argExpr = args.get(i).getValue();
+            if (argExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, argExpr.getSourceLocation(),
+                        "Function " + functionId.getNamespace() + "." + functionId.getName()
+                                + " expects constant arguments while arg[" + i + "] is of type "
+                                + argExpr.getExpressionTag());
             }
         }
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
-            throw new AlgebricksException("No positional variables are allowed over datasource functions");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over datasource functions");
         }
         FunctionDataSource datasource = toDatasource(context, f);
         List<LogicalVariable> variables = new ArrayList<>();
         variables.add(unnest.getVariable());
         DataSourceScanOperator scan = new DataSourceScanOperator(variables, datasource);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
index 89bd115..259e6d7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.function;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -45,7 +47,8 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
         }
         return new StorageComponentsDatasource(context.getComputationNodeDomain(), dataset.getDatasetId());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 53d4f3f..977bbe3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.translator.RequestParameters;
+import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -64,7 +65,7 @@
     private static final long serialVersionUID = 1L;
     private static final Logger LOGGER = LogManager.getLogger();
     //TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2062
-    public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
+    public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(Long.MAX_VALUE);
     //TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2063
     public static final long DEFAULT_QUERY_CANCELLATION_WAIT_MILLIS = TimeUnit.MINUTES.toMillis(1);
     private final String requestNodeId;
@@ -131,6 +132,7 @@
             responseMsg.setResult(outWriter.toString());
             responseMsg.setMetadata(outMetadata);
             responseMsg.setStats(stats);
+            responseMsg.setExecutionPlans(translator.getExecutionPlans());
         } catch (AlgebricksException | HyracksException | TokenMgrError
                 | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
             // we trust that "our" exceptions are serializable and have a comprehensible error message
@@ -138,7 +140,7 @@
             responseMsg.setError(pe);
         } catch (Exception e) {
             GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "Unexpected exception", e);
-            responseMsg.setError(new Exception(e.toString()));
+            responseMsg.setError(e);
         }
         try {
             messageBroker.sendApplicationMessageToNC(responseMsg, requestNodeId);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
index 7475be4..94dd541 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
@@ -23,6 +23,7 @@
 import org.apache.asterix.common.messaging.api.INcAddressedMessage;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.asterix.translator.ExecutionPlans;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -39,6 +40,8 @@
 
     private Throwable error;
 
+    private ExecutionPlans executionPlans;
+
     public ExecuteStatementResponseMessage(long requestMessageId) {
         this.requestMessageId = requestMessageId;
     }
@@ -84,6 +87,14 @@
         this.stats = stats;
     }
 
+    public ExecutionPlans getExecutionPlans() {
+        return executionPlans;
+    }
+
+    public void setExecutionPlans(ExecutionPlans executionPlans) {
+        this.executionPlans = executionPlans;
+    }
+
     @Override
     public String toString() {
         return String.format("%s(id=%s): %d characters", getClass().getSimpleName(), requestMessageId,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
new file mode 100644
index 0000000..db34131
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.nc;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HaltCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+    public static final HaltCallback INSTANCE = new HaltCallback();
+
+    private HaltCallback() {
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler has failed", failure);
+        ExitUtil.halt(ExitUtil.EC_IO_SCHEDULER_FAILED);
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} has failed", t);
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            ExitUtil.halt(ExitUtil.EC_FLUSH_FAILED);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
index a012f1e..c1a81a3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
@@ -22,6 +22,7 @@
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.nio.channels.ClosedByInterruptException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -54,29 +55,36 @@
     }
 
     @Override
-    public synchronized void init(long lsn) throws HyracksDataException {
-        final List<IndexCheckpoint> checkpoints = getCheckpoints();
+    public synchronized void init(String lastComponentTimestamp, long lsn) throws HyracksDataException {
+        List<IndexCheckpoint> checkpoints;
+        try {
+            checkpoints = getCheckpoints();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
         if (!checkpoints.isEmpty()) {
             LOGGER.warn(() -> "Checkpoints found on initializing: " + indexPath);
             delete();
         }
-        IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(lsn);
+        IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(lastComponentTimestamp, lsn);
         persist(firstCheckpoint);
     }
 
     @Override
-    public synchronized void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException {
+    public synchronized void replicated(String componentTimestamp, long masterLsn, long componentId)
+            throws HyracksDataException {
         final Long localLsn = getLatest().getMasterNodeFlushMap().get(masterLsn);
         if (localLsn == null) {
             throw new IllegalStateException("Component flushed before lsn mapping was received");
         }
-        flushed(componentTimestamp, localLsn);
+        flushed(componentTimestamp, localLsn, componentId);
     }
 
     @Override
-    public synchronized void flushed(String componentTimestamp, long lsn) throws HyracksDataException {
+    public synchronized void flushed(String componentTimestamp, long lsn, long componentId)
+            throws HyracksDataException {
         final IndexCheckpoint latest = getLatest();
-        IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentTimestamp);
+        IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentTimestamp, componentId);
         persist(nextCheckpoint);
         deleteHistory(nextCheckpoint.getId(), HISTORY_CHECKPOINTS);
     }
@@ -85,19 +93,19 @@
     public synchronized void masterFlush(long masterLsn, long localLsn) throws HyracksDataException {
         final IndexCheckpoint latest = getLatest();
         latest.getMasterNodeFlushMap().put(masterLsn, localLsn);
-        final IndexCheckpoint next =
-                IndexCheckpoint.next(latest, latest.getLowWatermark(), latest.getValidComponentTimestamp());
+        final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+                latest.getValidComponentTimestamp(), latest.getLastComponentId());
         persist(next);
         notifyAll();
     }
 
     @Override
-    public synchronized long getLowWatermark() {
+    public synchronized long getLowWatermark() throws HyracksDataException {
         return getLatest().getLowWatermark();
     }
 
     @Override
-    public synchronized boolean isFlushed(long masterLsn) {
+    public synchronized boolean isFlushed(long masterLsn) throws HyracksDataException {
         if (masterLsn == BULKLOAD_LSN) {
             return true;
         }
@@ -110,18 +118,28 @@
     }
 
     @Override
-    public Optional<String> getValidComponentTimestamp() {
-        final String validComponentTimestamp = getLatest().getValidComponentTimestamp();
+    public Optional<String> getValidComponentTimestamp() throws HyracksDataException {
+        String validComponentTimestamp = getLatest().getValidComponentTimestamp();
         return validComponentTimestamp != null ? Optional.of(validComponentTimestamp) : Optional.empty();
     }
 
     @Override
-    public int getCheckpointCount() {
-        return getCheckpoints().size();
+    public int getCheckpointCount() throws HyracksDataException {
+        try {
+            return getCheckpoints().size();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
     }
 
-    private IndexCheckpoint getLatest() {
-        final List<IndexCheckpoint> checkpoints = getCheckpoints();
+    @Override
+    public synchronized IndexCheckpoint getLatest() throws HyracksDataException {
+        List<IndexCheckpoint> checkpoints;
+        try {
+            checkpoints = getCheckpoints();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
         if (checkpoints.isEmpty()) {
             throw new IllegalStateException("Couldn't find any checkpoints for resource: " + indexPath);
         }
@@ -129,13 +147,34 @@
         return checkpoints.get(0);
     }
 
-    private List<IndexCheckpoint> getCheckpoints() {
+    @Override
+    public synchronized void setLastComponentId(long componentId) throws HyracksDataException {
+        final IndexCheckpoint latest = getLatest();
+        final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+                latest.getValidComponentTimestamp(), componentId);
+        persist(next);
+    }
+
+    @Override
+    public synchronized void advanceValidComponentTimestamp(String timestamp) throws HyracksDataException {
+        final IndexCheckpoint latest = getLatest();
+        if (latest.getValidComponentTimestamp() == null
+                || timestamp.compareTo(latest.getValidComponentTimestamp()) > 0) {
+            final IndexCheckpoint next =
+                    IndexCheckpoint.next(latest, latest.getLowWatermark(), timestamp, latest.getLastComponentId());
+            persist(next);
+        }
+    }
+
+    private List<IndexCheckpoint> getCheckpoints() throws ClosedByInterruptException {
         List<IndexCheckpoint> checkpoints = new ArrayList<>();
         final File[] checkpointFiles = indexPath.toFile().listFiles(CHECKPOINT_FILE_FILTER);
         if (checkpointFiles != null) {
             for (File checkpointFile : checkpointFiles) {
                 try {
                     checkpoints.add(read(checkpointFile.toPath()));
+                } catch (ClosedByInterruptException e) {
+                    throw e;
                 } catch (IOException e) {
                     LOGGER.warn(() -> "Couldn't read index checkpoint file: " + checkpointFile, e);
                 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
index 19ad8f6..e0b3105 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 
@@ -53,14 +54,10 @@
 
     private IndexCheckpointManager create(ResourceReference ref) {
         try {
-            final Path indexPath = getIndexPath(ref);
+            final Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
             return new IndexCheckpointManager(indexPath);
         } catch (HyracksDataException e) {
             throw new IllegalStateException(e);
         }
     }
-
-    private Path getIndexPath(ResourceReference indexRef) throws HyracksDataException {
-        return ioManager.resolve(indexRef.getRelativePath().toString()).getFile().toPath();
-    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 288e5f2..c3201c3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -52,7 +52,6 @@
 import org.apache.asterix.common.context.DatasetLifecycleManager;
 import org.apache.asterix.common.context.DatasetMemoryManager;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
@@ -61,6 +60,7 @@
 import org.apache.asterix.common.storage.IReplicaManager;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.file.StorageComponentProvider;
@@ -169,7 +169,7 @@
     }
 
     @Override
-    public void initialize(boolean initialRun) throws IOException, ACIDException {
+    public void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun) throws IOException {
         ioManager = getServiceContext().getIoManager();
         threadExecutor =
                 MaintainedThreadNameExecutorService.newCachedThreadPool(getServiceContext().getThreadFactory());
@@ -177,20 +177,14 @@
         IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
-
-        AsynchronousScheduler.INSTANCE.init(getServiceContext().getThreadFactory());
-        lsmIOScheduler = AsynchronousScheduler.INSTANCE;
-
+        lsmIOScheduler = new AsynchronousScheduler(getServiceContext().getThreadFactory(), HaltCallback.INSTANCE);
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
         indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
-
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
                 new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
-
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
-
-        txnSubsystem = new TransactionSubsystem(this);
+        txnSubsystem = new TransactionSubsystem(this, recoveryManagerFactory);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
         if (initialRun || systemState == SystemState.PERMANENT_DATA_LOSS) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index d4e652d..5e8a5e8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -48,7 +48,7 @@
 import org.apache.asterix.common.context.IndexInfo;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -77,6 +77,8 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -102,14 +104,14 @@
     private static final String RECOVERY_FILES_DIR_NAME = "recovery_temp";
     private Map<Long, JobEntityCommits> jobId2WinnerEntitiesMap = null;
     private final long cachedEntityCommitsPerJobSize;
-    private final PersistentLocalResourceRepository localResourceRepository;
+    protected final PersistentLocalResourceRepository localResourceRepository;
     private final ICheckpointManager checkpointManager;
     private SystemState state;
-    private final INCServiceContext serviceCtx;
-    private final INcApplicationContext appCtx;
+    protected final INCServiceContext serviceCtx;
+    protected final INcApplicationContext appCtx;
     private static final TxnId recoveryTxnId = new TxnId(-1);
 
-    public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
+    public RecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem) {
         this.serviceCtx = serviceCtx;
         this.txnSubsystem = txnSubsystem;
         this.appCtx = txnSubsystem.getApplicationContext();
@@ -225,6 +227,7 @@
                     break;
                 case LogType.FLUSH:
                 case LogType.WAIT:
+                case LogType.WAIT_FOR_FLUSHES:
                 case LogType.MARKER:
                 case LogType.FILTER:
                     break;
@@ -392,7 +395,6 @@
                                 logRecord = logReader.next();
                                 continue;
                             }
-                            idGenerator.refresh();
                             DatasetInfo dsInfo = datasetLifecycleManager.getDatasetInfo(datasetId);
                             // we only need to flush open indexes here (opened by previous update records)
                             // if an index has no ongoing updates, then it's memory component must be empty
@@ -401,24 +403,15 @@
                                 if (iInfo.isOpen() && iInfo.getPartition() == partition) {
                                     maxDiskLastLsn = resourceId2MaxLSNMap.get(iInfo.getResourceId());
                                     index = iInfo.getIndex();
-                                    AbstractLSMIOOperationCallback ioCallback =
-                                            (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
                                     if (logRecord.getLSN() > maxDiskLastLsn
                                             && !index.isCurrentMutableComponentEmpty()) {
                                         // schedule flush
-                                        ioCallback.updateLastLSN(logRecord.getLSN());
                                         redoFlush(index, logRecord);
                                         redoCount++;
                                     } else {
-                                        if (index.isMemoryComponentsAllocated()) {
-                                            // if the memory component has been allocated, we
-                                            // force it to receive the same Id
-                                            index.getCurrentMemoryComponent().resetId(idGenerator.getId(), true);
-                                        } else {
-                                            // otherwise, we refresh the id stored in ioCallback
-                                            // to ensure the memory component receives correct Id upon activation
-                                            ioCallback.forceRefreshNextId();
-                                        }
+                                        // otherwise, do nothing since this component had no records when flush was
+                                        // scheduled.. TODO: update checkpoint file? and do the
+                                        // lsn checks from the checkpoint file
                                     }
                                 }
                             }
@@ -428,6 +421,7 @@
                     case LogType.ENTITY_COMMIT:
                     case LogType.ABORT:
                     case LogType.WAIT:
+                    case LogType.WAIT_FOR_FLUSHES:
                     case LogType.MARKER:
                         //do nothing
                         break;
@@ -473,10 +467,10 @@
         long minFirstLSN = logMgr.getAppendLSN();
         if (!openIndexList.isEmpty()) {
             for (IIndex index : openIndexList) {
-                AbstractLSMIOOperationCallback ioCallback =
-                        (AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
+                LSMIOOperationCallback ioCallback =
+                        (LSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
                 if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
-                    firstLSN = ioCallback.getFirstLSN();
+                    firstLSN = ioCallback.getPersistenceLsn();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
                 }
             }
@@ -684,6 +678,7 @@
                     case LogType.FLUSH:
                     case LogType.FILTER:
                     case LogType.WAIT:
+                    case LogType.WAIT_FOR_FLUSHES:
                     case LogType.MARKER:
                         //ignore
                         break;
@@ -824,10 +819,15 @@
     }
 
     private static void redoFlush(ILSMIndex index, ILogRecord logRecord) throws HyracksDataException {
+        long flushLsn = logRecord.getLSN();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        accessor.getOpContext().setParameters(flushMap);
         long minId = logRecord.getFlushingComponentMinId();
         long maxId = logRecord.getFlushingComponentMaxId();
         ILSMComponentId id = new LSMComponentId(minId, maxId);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, index.getCurrentMemoryComponent().getId());
         if (!index.getDiskComponents().isEmpty()) {
             ILSMDiskComponent diskComponent = index.getDiskComponents().get(0);
             ILSMComponentId maxDiskComponentId = diskComponent.getId();
@@ -837,7 +837,17 @@
             }
         }
         index.getCurrentMemoryComponent().resetId(id, true);
-        accessor.scheduleFlush(index.getIOOperationCallback());
+        ILSMIOOperation flush = accessor.scheduleFlush();
+        try {
+            flush.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(flush.getFailure());
+        }
+        index.resetCurrentComponentIndex();
     }
 
     private class JobEntityCommits {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index 8158096..79c87c0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.common.transactions.ILockManager;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.IRecoveryManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.transaction.management.service.locking.ConcurrentLockManager;
@@ -59,7 +60,7 @@
     private long profilerEntityCommitLogCount = 0;
     private EntityCommitProfiler ecp;
 
-    public TransactionSubsystem(INcApplicationContext appCtx) {
+    public TransactionSubsystem(INcApplicationContext appCtx, IRecoveryManagerFactory recoveryManagerFactory) {
         this.appCtx = appCtx;
         this.id = appCtx.getServiceContext().getNodeId();
         this.txnProperties = appCtx.getTransactionProperties();
@@ -78,7 +79,7 @@
         }
 
         this.logManager = replicationEnabled ? new LogManagerWithReplication(this) : new LogManager(this);
-        this.recoveryManager = new RecoveryManager(this, appCtx.getServiceContext());
+        this.recoveryManager = recoveryManagerFactory.createRecoveryManager(appCtx.getServiceContext(), this);
         if (txnProperties.isCommitProfilerEnabled()) {
             ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
             ((ExecutorService) appCtx.getThreadExecutor()).submit(ecp);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index e983763..bc6fff1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -156,6 +156,8 @@
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledUpsertStatement;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansHtmlPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorContext;
@@ -185,6 +187,7 @@
 import org.apache.hyracks.api.dataset.IHyracksDataset;
 import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.UnmanagedFileSplit;
 import org.apache.hyracks.api.job.JobFlag;
@@ -400,7 +403,8 @@
                                 resultSetIdCounter);
                         break;
                     default:
-                        throw new CompilationException("Unknown function");
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
+                                "Unexpected statement: " + stmt.getKind());
                 }
             }
         } finally {
@@ -430,6 +434,7 @@
     protected Dataverse handleUseDataverseStatement(MetadataProvider metadataProvider, Statement stmt)
             throws Exception {
         DataverseDecl dvd = (DataverseDecl) stmt;
+        SourceLocation sourceLoc = dvd.getSourceLocation();
         String dvName = dvd.getDataverseName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -437,13 +442,13 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
             if (dv == null) {
-                throw new MetadataException("Unknown dataverse " + dvName);
+                throw new MetadataException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dvName);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return dv;
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.METADATA_ERROR, e, sourceLoc, e.toString());
         } finally {
             metadataProvider.getLocks().unlock();
         }
@@ -464,7 +469,7 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
+                    throw new CompilationException(ErrorCode.DATAVERSE_EXISTS, stmt.getSourceLocation(), dvName);
                 }
             }
             MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(),
@@ -480,31 +485,36 @@
 
     protected static void validateCompactionPolicy(String compactionPolicy,
             Map<String, String> compactionPolicyProperties, MetadataTransactionContext mdTxnCtx,
-            boolean isExternalDataset) throws CompilationException, Exception {
+            boolean isExternalDataset, SourceLocation sourceLoc) throws CompilationException, Exception {
         CompactionPolicy compactionPolicyEntity = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
                 MetadataConstants.METADATA_DATAVERSE_NAME, compactionPolicy);
         if (compactionPolicyEntity == null) {
-            throw new CompilationException("Unknown compaction policy: " + compactionPolicy);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Unknown compaction policy: " + compactionPolicy);
         }
         String compactionPolicyFactoryClassName = compactionPolicyEntity.getClassName();
         ILSMMergePolicyFactory mergePolicyFactory =
                 (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
         if (isExternalDataset && mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
-            throw new CompilationException("The correlated-prefix merge policy cannot be used with external dataset.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "The correlated-prefix merge policy cannot be used with external dataset.");
         }
         if (compactionPolicyProperties == null) {
             if (mergePolicyFactory.getName().compareTo("no-merge") != 0) {
-                throw new CompilationException("Compaction policy properties are missing.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Compaction policy properties are missing.");
             }
         } else {
             for (Map.Entry<String, String> entry : compactionPolicyProperties.entrySet()) {
                 if (!mergePolicyFactory.getPropertiesNames().contains(entry.getKey())) {
-                    throw new CompilationException("Invalid compaction policy property: " + entry.getKey());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Invalid compaction policy property: " + entry.getKey());
                 }
             }
             for (String p : mergePolicyFactory.getPropertiesNames()) {
                 if (!compactionPolicyProperties.containsKey(p)) {
-                    throw new CompilationException("Missing compaction policy property: " + p);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Missing compaction policy property: " + p);
                 }
             }
         }
@@ -514,6 +524,7 @@
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws CompilationException, Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
         DatasetDecl dd = (DatasetDecl) stmt;
+        SourceLocation sourceLoc = dd.getSourceLocation();
         String dataverseName = getActiveDataverse(dd.getDataverse());
         String datasetName = dd.getName().getValue();
         DatasetType dsType = dd.getDatasetType();
@@ -542,28 +553,30 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A dataset with this name " + datasetName + " already exists.");
+                    throw new CompilationException(ErrorCode.DATASET_EXISTS, sourceLoc, datasetName);
                 }
             }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     itemTypeDataverseName, itemTypeName);
             if (dt == null) {
-                throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
+                throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, itemTypeName);
             }
             String ngName = ngNameId != null ? ngNameId.getValue()
-                    : configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider);
+                    : configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider,
+                            sourceLoc);
 
             if (compactionPolicy == null) {
                 compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
                 compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
             } else {
-                validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false);
+                validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false, sourceLoc);
             }
             switch (dd.getDatasetType()) {
                 case INTERNAL:
                     IAType itemType = dt.getDatatype();
                     if (itemType.getTypeTag() != ATypeTag.OBJECT) {
-                        throw new AlgebricksException("Dataset type has to be a record type.");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Dataset type has to be a record type.");
                     }
 
                     IAType metaItemType = null;
@@ -571,7 +584,8 @@
                         metaItemType = metadataProvider.findType(metaItemTypeDataverseName, metaItemTypeName);
                     }
                     if (metaItemType != null && metaItemType.getTypeTag() != ATypeTag.OBJECT) {
-                        throw new AlgebricksException("Dataset meta type has to be a record type.");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Dataset meta type has to be a record type.");
                     }
                     ARecordType metaRecType = (ARecordType) metaItemType;
 
@@ -582,11 +596,11 @@
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
-                            metaRecType, partitioningExprs, keySourceIndicators, autogenerated);
+                            metaRecType, partitioningExprs, keySourceIndicators, autogenerated, sourceLoc);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
-                        ValidateUtil.validateFilterField(aRecordType, filterField);
+                        ValidateUtil.validateFilterField(aRecordType, filterField, sourceLoc);
                     }
                     if (compactionPolicy == null && filterField != null) {
                         // If the dataset has a filter and the user didn't specify a merge
@@ -607,7 +621,8 @@
                             new ExternalDatasetDetails(adapter, properties, new Date(), TransactionState.COMMIT);
                     break;
                 default:
-                    throw new CompilationException("Unknown datatype " + dd.getDatasetType());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unknown dataset type " + dd.getDatasetType());
             }
 
             // #. initialize DatasetIdFactory if it is not initialized.
@@ -692,8 +707,8 @@
         }
     }
 
-    protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset)
-            throws CompilationException {
+    protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset,
+            SourceLocation sourceLoc) throws CompilationException {
         StringBuilder builder = null;
         ActiveNotificationHandler activeEventHandler =
                 (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -707,13 +722,15 @@
             }
         }
         if (builder != null) {
-            throw new CompilationException("Dataset " + dataset.getDataverseName() + "." + dataset.getDatasetName()
-                    + " is currently being " + "fed into by the following active entities.\n" + builder.toString());
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Dataset " + dataset.getDataverseName() + "." + dataset.getDatasetName() + " is currently being "
+                            + "fed into by the following active entities.\n" + builder.toString());
         }
     }
 
     protected static String configureNodegroupForDataset(ICcApplicationContext appCtx, Map<String, String> hints,
-            String dataverseName, String datasetName, MetadataProvider metadataProvider) throws Exception {
+            String dataverseName, String datasetName, MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws Exception {
         IClusterStateManager csm = appCtx.getClusterStateManager();
         Set<String> allNodes = csm.getParticipantNodes(true);
         Set<String> selectedNodes = new LinkedHashSet<>();
@@ -726,7 +743,7 @@
                     hints.get(DatasetNodegroupCardinalityHint.NAME));
             boolean valid = validation.first;
             if (!valid) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Incorrect use of hint '" + DatasetNodegroupCardinalityHint.NAME + "': " + validation.second);
             } else {
                 nodegroupCardinality = Integer.parseInt(hints.get(DatasetNodegroupCardinalityHint.NAME));
@@ -742,6 +759,7 @@
     public void handleCreateIndexStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+        SourceLocation sourceLoc = stmtCreateIndex.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
@@ -756,8 +774,8 @@
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
 
             index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
@@ -767,7 +785,7 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("An index with this name " + indexName + " already exists.");
+                    throw new CompilationException(ErrorCode.INDEX_EXISTS, sourceLoc, indexName);
                 }
             }
             // can't create secondary primary index on an external dataset
@@ -814,11 +832,11 @@
                     fieldType = subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size()));
                 } else {
                     if (!stmtCreateIndex.isEnforced() && stmtCreateIndex.getIndexType() != IndexType.BTREE) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc,
                                 stmtCreateIndex.getIndexType());
                     }
                     if (stmtCreateIndex.isEnforced() && !fieldExpr.second.isUnknownable()) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL, sourceLoc,
                                 String.valueOf(fieldExpr.first));
                     }
                     // don't allow creating an enforced index on a closed-type field, fields that
@@ -826,15 +844,16 @@
                     // get the field type, if it's not null, then the field is closed-type
                     if (stmtCreateIndex.isEnforced()
                             && subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD, sourceLoc,
                                 String.valueOf(fieldExpr.first));
                     }
                     if (!isOpen) {
-                        throw new AlgebricksException("Typed index on \"" + fieldExpr.first
-                                + "\" field could be created only for open datatype");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Typed index on \""
+                                + fieldExpr.first + "\" field could be created only for open datatype");
                     }
                     if (stmtCreateIndex.hasMetaField()) {
-                        throw new AlgebricksException("Typed open index can only be created on the record part");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Typed open index can only be created on the record part");
                     }
                     Map<TypeSignature, IAType> typeMap =
                             TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second.getType(), indexName, dataverseName);
@@ -843,15 +862,15 @@
                     overridesFieldTypes = true;
                 }
                 if (fieldType == null) {
-                    throw new AlgebricksException(
-                            "Unknown type " + (fieldExpr.second == null ? fieldExpr.first : fieldExpr.second));
+                    throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, fieldExpr.second == null
+                            ? String.valueOf(fieldExpr.first) : String.valueOf(fieldExpr.second));
                 }
 
                 // try to add the key & its source to the set of keys, if key couldn't be added,
                 // there is a duplicate
                 if (!indexKeysSet
                         .add(new Pair<>(fieldExpr.first, stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
-                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
+                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD, sourceLoc,
                             String.valueOf(fieldExpr.first));
                 }
 
@@ -879,9 +898,10 @@
 
                     // If it is not a fixed length
                     if (typeTrait.getFixedLength() < 0) {
-                        throw new AlgebricksException("The keyword or ngram index -" + indexName
-                                + " cannot be created on the dataset -" + datasetName
-                                + " due to its variable-length primary key field - " + partitioningKey);
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The keyword or ngram index -" + indexName + " cannot be created on the dataset -"
+                                        + datasetName + " due to its variable-length primary key field - "
+                                        + partitioningKey);
                     }
 
                 }
@@ -890,14 +910,14 @@
             Index newIndex = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
                     indexFields, keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(),
                     overridesFieldTypes, stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
-            doCreateIndex(hcc, metadataProvider, ds, newIndex, jobFlags);
+            doCreateIndex(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     public static void doCreateIndex(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Dataset ds,
-            Index index, EnumSet<JobFlag> jobFlags) throws Exception {
+            Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc) throws Exception {
         ProgressState progress = ProgressState.NO_PROGRESS;
         boolean bActiveTxn = true;
         Index filesIndex = null;
@@ -910,18 +930,19 @@
         try {
             index.setPendingOp(MetadataUtil.PENDING_ADD_OP);
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
-                validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), ds);
+                validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), ds, sourceLoc);
             } else {
                 // External dataset
                 // Check if the dataset is indexible
                 if (!ExternalIndexingOperations.isIndexible((ExternalDatasetDetails) ds.getDatasetDetails())) {
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                             "dataset using " + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter()
                                     + " Adapter can't be indexed");
                 }
                 // Check if the name of the index is valid
                 if (!ExternalIndexingOperations.isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                    throw new AlgebricksException("external dataset index name is invalid");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "external dataset index name is invalid");
                 }
 
                 // Check if the files index exist
@@ -961,7 +982,7 @@
                     spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot,
                             metadataProvider);
                     if (spec == null) {
-                        throw new CompilationException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                                 "Failed to create job spec for replicating Files Index For external dataset");
                     }
                     filesIndexReplicated = true;
@@ -977,20 +998,21 @@
                     if (existingIndex.getKeyFieldNames().equals(index.getKeyFieldNames())
                             && !existingIndex.getKeyFieldTypes().equals(index.getKeyFieldTypes())
                             && existingIndex.isEnforced()) {
-                        throw new CompilationException("Cannot create index " + index.getIndexName()
-                                + " , enforced index " + existingIndex.getIndexName() + " on field \""
-                                + StringUtils.join(index.getKeyFieldNames(), ',') + "\" is already defined with type \""
-                                + existingIndex.getKeyFieldTypes() + "\"");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Cannot create index "
+                                + index.getIndexName() + " , enforced index " + existingIndex.getIndexName()
+                                + " on field \"" + StringUtils.join(index.getKeyFieldNames(), ',')
+                                + "\" is already defined with type \"" + existingIndex.getKeyFieldTypes() + "\"");
                     }
                 }
             }
             // #. add a new index with PendingAddOp
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
             // #. prepare to create the index artifact in NC.
-            spec = IndexUtil.buildSecondaryIndexCreationJobSpec(ds, index, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexCreationJobSpec(ds, index, metadataProvider, sourceLoc);
             if (spec == null) {
-                throw new CompilationException("Failed to create job spec for creating index '" + ds.getDatasetName()
-                        + "." + index.getIndexName() + "'");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
+                                + index.getIndexName() + "'");
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1013,7 +1035,7 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
             // #. load data into the index in NC.
-            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
@@ -1072,7 +1094,7 @@
                 bActiveTxn = true;
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
                 try {
-                    JobSpecification jobSpec = IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds);
+                    JobSpecification jobSpec = IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     runJob(hcc, jobSpec, jobFlags);
@@ -1147,6 +1169,7 @@
 
     protected void handleCreateTypeStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         TypeDecl stmtCreateType = (TypeDecl) stmt;
+        SourceLocation sourceLoc = stmtCreateType.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
         String typeName = stmtCreateType.getIdent().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1156,16 +1179,17 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
-                throw new AlgebricksException("Unknown dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
             }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt != null) {
                 if (!stmtCreateType.getIfNotExists()) {
-                    throw new AlgebricksException("A datatype with this name " + typeName + " already exists.");
+                    throw new CompilationException(ErrorCode.TYPE_EXISTS, sourceLoc, typeName);
                 }
             } else {
                 if (BuiltinTypeMap.getBuiltinType(typeName) != null) {
-                    throw new AlgebricksException("Cannot redefine builtin type " + typeName + ".");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Cannot redefine builtin type " + typeName + ".");
                 } else {
                     Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx,
                             stmtCreateType.getTypeDef(), stmtCreateType.getIdent().getValue(), dataverseName);
@@ -1186,9 +1210,10 @@
     protected void handleDataverseDropStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String dataverseName = stmtDelete.getDataverseName().getValue();
         if (dataverseName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
-            throw new HyracksDataException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME + " dataverse can't be dropped");
         }
 
@@ -1205,7 +1230,7 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("There is no dataverse with this name " + dataverseName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
                 }
             }
             // # check whether any function in current dataverse is being used by others
@@ -1213,7 +1238,7 @@
                     MetadataManager.INSTANCE.getDataverseFunctions(mdTxnCtx, dataverseName);
             for (Function function : functionsInDataverse) {
                 if (isFunctionUsed(mdTxnCtx, function.getSignature(), dataverseName)) {
-                    throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE,
+                    throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, sourceLoc,
                             function.getDataverseName() + "." + function.getName() + "@" + function.getArity());
                 }
             }
@@ -1234,7 +1259,7 @@
                     mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                     bActiveTxn = true;
                     metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                    doDropFeed(hcc, metadataProvider, feedListener.getFeed());
+                    doDropFeed(hcc, metadataProvider, feedListener.getFeed(), sourceLoc);
                     MetadataManager.INSTANCE.commitTransaction(metadataProvider.getMetadataTxnContext());
                     bActiveTxn = false;
                 }
@@ -1252,7 +1277,7 @@
                     List<Index> indexes =
                             MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (Index index : indexes) {
-                        jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset));
+                        jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
                     }
                 } else {
                     // External dataset
@@ -1263,8 +1288,8 @@
                             jobsToExecute.add(
                                     ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, dataset));
                         } else {
-                            jobsToExecute
-                                    .add(IndexUtil.buildDropIndexJobSpec(indexes.get(k), metadataProvider, dataset));
+                            jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(k), metadataProvider, dataset,
+                                    sourceLoc));
                         }
                     }
                     ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(dataset);
@@ -1351,19 +1376,21 @@
     public void handleDatasetDropStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DropDatasetStatement stmtDelete = (DropDatasetStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
         String datasetName = stmtDelete.getDatasetName().getValue();
         MetadataLockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
-            doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true);
+            doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true, sourceLoc);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     public static void doDropDataset(String dataverseName, String datasetName, MetadataProvider metadataProvider,
-            boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup) throws Exception {
+            boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup,
+            SourceLocation sourceLoc) throws Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
         MutableObject<MetadataTransactionContext> mdTxnCtx =
                 new MutableObject<>(MetadataManager.INSTANCE.beginTransaction());
@@ -1377,10 +1404,12 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
                     return;
                 } else {
-                    throw new AsterixException(ErrorCode.NO_DATASET_WITH_NAME, dataverseName, datasetName);
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                            dataverseName);
                 }
             }
-            ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup);
+            ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup,
+                    sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
         } catch (Exception e) {
             if (bActiveTxn.booleanValue()) {
@@ -1423,6 +1452,7 @@
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
 
         IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
+        SourceLocation sourceLoc = stmtIndexDrop.getSourceLocation();
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
         String dataverseName = getActiveDataverse(stmtIndexDrop.getDataverseName());
         String indexName = stmtIndexDrop.getIndexName().getValue();
@@ -1438,8 +1468,8 @@
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             ActiveNotificationHandler activeEventHandler =
                     (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -1454,7 +1484,7 @@
                 }
             }
             if (builder != null) {
-                throw new CompilationException("Dataset" + datasetName
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Dataset" + datasetName
                         + " is currently being fed into by the following active entities: " + builder.toString());
             }
 
@@ -1465,12 +1495,12 @@
                         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                         return;
                     } else {
-                        throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                        throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName);
                     }
                 }
-                ensureNonPrimaryIndexDrop(index);
+                ensureNonPrimaryIndexDrop(index, sourceLoc);
                 // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds));
+                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
 
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
@@ -1505,14 +1535,15 @@
                         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                         return;
                     } else {
-                        throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                        throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName);
                     }
                 } else if (ExternalIndexingOperations.isFileIndex(index)) {
-                    throw new AlgebricksException("Dropping a dataset's files index is not allowed.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Dropping a dataset's files index is not allowed.");
                 }
-                ensureNonPrimaryIndexDrop(index);
+                ensureNonPrimaryIndexDrop(index, sourceLoc);
                 // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds));
+                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
                 List<Index> datasetIndexes =
                         MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                 if (datasetIndexes.size() == 2) {
@@ -1614,8 +1645,8 @@
     }
 
     protected void handleTypeDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
-
         TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
+        SourceLocation sourceLoc = stmtTypeDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtTypeDrop.getDataverseName());
         String typeName = stmtTypeDrop.getTypeName().getValue();
 
@@ -1627,7 +1658,7 @@
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt == null) {
                 if (!stmtTypeDrop.getIfExists()) {
-                    throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, typeName);
                 }
             } else {
                 MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, dataverseName, typeName);
@@ -1643,6 +1674,7 @@
 
     protected void handleNodegroupDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String nodegroupName = stmtDelete.getNodeGroupName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -1651,7 +1683,7 @@
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
             if (ng == null) {
                 if (!stmtDelete.getIfExists()) {
-                    throw new AlgebricksException("There is no nodegroup with this name " + nodegroupName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_NODEGROUP, sourceLoc, nodegroupName);
                 }
             } else {
                 MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodegroupName, false);
@@ -1668,6 +1700,7 @@
 
     protected void handleCreateFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         FunctionSignature signature = cfs.getFunctionSignature();
         String dataverse = getActiveDataverseName(signature.getNamespace());
         signature.setNamespace(dataverse);
@@ -1679,12 +1712,13 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
             if (dv == null) {
-                throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverse);
             }
 
             //Check whether the function is use-able
             metadataProvider.setDefaultDataverse(dv);
             Query wrappedQuery = new Query(false);
+            wrappedQuery.setSourceLocation(sourceLoc);
             wrappedQuery.setBody(cfs.getFunctionBodyExpression());
             wrappedQuery.setTopLevel(false);
             List<VarIdentifier> varIds = new ArrayList<>();
@@ -1736,6 +1770,7 @@
 
     protected void handleFunctionDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDropFunction.getSourceLocation();
         FunctionSignature signature = stmtDropFunction.getFunctionSignature();
         signature.setNamespace(getActiveDataverseName(signature.getNamespace()));
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1745,9 +1780,9 @@
         try {
             Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
             if (function == null && !stmtDropFunction.getIfExists()) {
-                throw new AlgebricksException("Unknonw function " + signature);
+                throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, signature);
             } else if (isFunctionUsed(mdTxnCtx, signature, null)) {
-                throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, signature);
+                throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, sourceLoc, signature);
             } else {
                 MetadataManager.INSTANCE.dropFunction(mdTxnCtx, signature);
             }
@@ -1774,7 +1809,9 @@
             CompiledLoadFromFileStatement cls =
                     new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
                             loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
+            cls.setSourceLocation(stmt.getSourceLocation());
             JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls);
+            afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
             if (spec != null) {
@@ -1865,7 +1902,9 @@
             CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
                     stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
                     stmtDelete.getQuery());
+            clfrqs.setSourceLocation(stmt.getSourceLocation());
             JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs);
+            afterCompile();
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1901,6 +1940,7 @@
     private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
             MetadataProvider metadataProvider, InsertStatement insertUpsert)
             throws RemoteException, AlgebricksException, ACIDException {
+        SourceLocation sourceLoc = insertUpsert.getSourceLocation();
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata
         // transaction)
@@ -1916,14 +1956,17 @@
                 clfrqs = new CompiledInsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
                         rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
                         rewrittenInsertUpsert.getReturnExpression());
+                clfrqs.setSourceLocation(insertUpsert.getSourceLocation());
                 break;
             case UPSERT:
                 clfrqs = new CompiledUpsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
                         rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
                         rewrittenInsertUpsert.getReturnExpression());
+                clfrqs.setSourceLocation(insertUpsert.getSourceLocation());
                 break;
             default:
-                throw new AlgebricksException("Unsupported statement type " + rewrittenInsertUpsert.getKind());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported statement type " + rewrittenInsertUpsert.getKind());
         }
         // Insert/upsert statement compilation (happens under the same ongoing metadata
         // transaction)
@@ -1933,6 +1976,7 @@
 
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFeedStatement cfs = (CreateFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1947,7 +1991,8 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A feed with this name " + feedName + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A feed with this name " + feedName + " already exists.");
                 }
             }
             feed = new Feed(dataverseName, feedName, cfs.getConfiguration());
@@ -1969,6 +2014,7 @@
         FeedPolicyEntity newPolicy = null;
         MetadataTransactionContext mdTxnCtx = null;
         CreateFeedPolicyStatement cfps = (CreateFeedPolicyStatement) stmt;
+        SourceLocation sourceLoc = cfps.getSourceLocation();
         dataverse = getActiveDataverse(null);
         policy = cfps.getPolicyName();
         MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
@@ -1983,7 +2029,8 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A policy with this name " + policy + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A policy with this name " + policy + " already exists.");
                 }
             }
             boolean extendingExisting = cfps.getSourcePolicyName() != null;
@@ -1995,7 +2042,8 @@
                     sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
                             MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
                     if (sourceFeedPolicy == null) {
-                        throw new AlgebricksException("Unknown policy " + cfps.getSourcePolicyName());
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Unknown policy " + cfps.getSourcePolicyName());
                     }
                 }
                 Map<String, String> policyProperties = sourceFeedPolicy.getProperties();
@@ -2007,7 +2055,8 @@
                     InputStream stream = new FileInputStream(cfps.getSourcePolicyFile());
                     prop.load(stream);
                 } catch (Exception e) {
-                    throw new AlgebricksException("Unable to read policy file" + cfps.getSourcePolicyFile(), e);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unable to read policy file" + cfps.getSourcePolicyFile(), e);
                 }
                 Map<String, String> policyProperties = new HashMap<>();
                 prop.forEach((key, value) -> policyProperties.put((String) key, (String) value));
@@ -2026,6 +2075,7 @@
     protected void handleDropFeedStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
+        SourceLocation sourceLoc = stmtFeedDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtFeedDrop.getDataverseName());
         String feedName = stmtFeedDrop.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2036,12 +2086,13 @@
             Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
             if (feed == null) {
                 if (!stmtFeedDrop.getIfExists()) {
-                    throw new AlgebricksException("There is no feed with this name " + feedName + ".");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "There is no feed with this name " + feedName + ".");
                 }
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
             }
-            doDropFeed(hcc, metadataProvider, feed);
+            doDropFeed(hcc, metadataProvider, feed, sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
@@ -2051,8 +2102,8 @@
         }
     }
 
-    protected void doDropFeed(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Feed feed)
-            throws Exception {
+    protected void doDropFeed(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Feed feed,
+            SourceLocation sourceLoc) throws Exception {
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
         EntityId feedId = feed.getFeedId();
         ActiveNotificationHandler activeNotificationHandler =
@@ -2060,7 +2111,7 @@
         ActiveEntityEventsListener listener =
                 (ActiveEntityEventsListener) activeNotificationHandler.getListener(feedId);
         if (listener != null && listener.getState() != ActivityState.STOPPED) {
-            throw new AlgebricksException("Feed " + feedId
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Feed " + feedId
                     + " is currently active and connected to the following dataset(s) \n" + listener.toString());
         } else if (listener != null) {
             listener.unregister();
@@ -2078,6 +2129,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
+        SourceLocation sourceLoc = stmtFeedPolicyDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtFeedPolicyDrop.getDataverseName());
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
         MetadataLockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -2086,7 +2138,8 @@
             FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName, policyName);
             if (feedPolicy == null) {
                 if (!stmtFeedPolicyDrop.getIfExists()) {
-                    throw new AlgebricksException("Unknown policy " + policyName + " in dataverse " + dataverseName);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unknown policy " + policyName + " in dataverse " + dataverseName);
                 }
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
@@ -2104,6 +2157,7 @@
     private void handleStartFeedStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         StartFeedStatement sfs = (StartFeedStatement) stmt;
+        SourceLocation sourceLoc = sfs.getSourceLocation();
         String dataverseName = getActiveDataverse(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2120,7 +2174,7 @@
             List<FeedConnection> feedConnections = MetadataManager.INSTANCE
                     .getFeedConections(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
             if (feedConnections.isEmpty()) {
-                throw new CompilationException(ErrorCode.FEED_START_FEED_WITHOUT_CONNECTION, feedName);
+                throw new CompilationException(ErrorCode.FEED_START_FEED_WITHOUT_CONNECTION, sourceLoc, feedName);
             }
             for (FeedConnection feedConnection : feedConnections) {
                 // what if the dataset is in a different dataverse
@@ -2157,6 +2211,7 @@
 
     private void handleStopFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         StopFeedStatement sfst = (StopFeedStatement) stmt;
+        SourceLocation sourceLoc = sfst.getSourceLocation();
         String dataverseName = getActiveDataverse(sfst.getDataverseName());
         String feedName = sfst.getFeedName().getValue();
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
@@ -2165,7 +2220,8 @@
         // Obtain runtime info from ActiveListener
         ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
         if (listener == null) {
-            throw new AlgebricksException("Feed " + feedName + " is not started.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Feed " + feedName + " is not started.");
         }
         MetadataLockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), entityId.getDataverse(),
                 entityId.getEntityName());
@@ -2179,6 +2235,7 @@
     private void handleConnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FeedConnection fc;
         ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName();
         String datasetName = cfs.getDatasetName().getValue();
@@ -2200,21 +2257,23 @@
                     metadataProvider.getMetadataTxnContext());
             FeedEventsListener listener = (FeedEventsListener) activeEventHandler.getListener(feed.getFeedId());
             if (listener != null && listener.isActive()) {
-                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, sourceLoc,
+                        feedName);
             }
             ARecordType outputType = FeedMetadataUtil.getOutputType(feed,
                     feed.getConfiguration().get(ExternalDataConstants.KEY_TYPE_NAME));
             List<FunctionSignature> appliedFunctions = cfs.getAppliedFunctions();
             for (FunctionSignature func : appliedFunctions) {
                 if (MetadataManager.INSTANCE.getFunction(mdTxnCtx, func) == null) {
-                    throw new CompilationException(ErrorCode.FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION,
+                    throw new CompilationException(ErrorCode.FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION, sourceLoc,
                             func.getName());
                 }
             }
             fc = MetadataManager.INSTANCE.getFeedConnection(metadataProvider.getMetadataTxnContext(), dataverseName,
                     feedName, datasetName);
             if (fc != null) {
-                throw new AlgebricksException("Feed" + feedName + " is already connected dataset " + datasetName);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Feed" + feedName + " is already connected dataset " + datasetName);
             }
             fc = new FeedConnection(dataverseName, feedName, datasetName, appliedFunctions, policyName, whereClauseBody,
                     outputType.getTypeName());
@@ -2234,6 +2293,7 @@
 
     protected void handleDisconnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
         String feedName = cfs.getFeedName().getValue();
@@ -2248,7 +2308,8 @@
             ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler
                     .getListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName));
             if (listener != null && listener.isActive()) {
-                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, sourceLoc,
+                        feedName);
             }
             FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, cfs.getDatasetName().getValue());
             FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
@@ -2256,11 +2317,12 @@
                     dataverseName, feedName, datasetName);
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new CompilationException("Dataset " + dataverseName + "." + datasetName + " doesn't exist");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             if (fc == null) {
-                throw new CompilationException("Feed " + feedName + " is currently not connected to "
-                        + cfs.getDatasetName().getValue() + ". Invalid operation!");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Feed " + feedName
+                        + " is currently not connected to " + cfs.getDatasetName().getValue() + ". Invalid operation!");
             }
             MetadataManager.INSTANCE.dropFeedConnection(mdTxnCtx, dataverseName, feedName, datasetName);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2278,6 +2340,7 @@
     protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
+        SourceLocation sourceLoc = compactStatement.getSourceLocation();
         String dataverseName = getActiveDataverse(compactStatement.getDataverseName());
         String datasetName = compactStatement.getDatasetName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2289,13 +2352,13 @@
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName + ".");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             // Prepare jobs to compact the datatset and its indexes
             List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
             if (indexes.isEmpty()) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Cannot compact the extrenal dataset " + datasetName + " because it has no indexes");
             }
             Dataverse dataverse =
@@ -2305,11 +2368,12 @@
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 for (Index index : indexes) {
                     if (index.isSecondaryIndex()) {
-                        jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, index, metadataProvider));
+                        jobsToExecute.add(
+                                IndexUtil.buildSecondaryIndexCompactJobSpec(ds, index, metadataProvider, sourceLoc));
                     }
                 }
             } else {
-                prepareCompactJobsForExternalDataset(indexes, ds, jobsToExecute, metadataProvider);
+                prepareCompactJobsForExternalDataset(indexes, ds, jobsToExecute, metadataProvider, sourceLoc);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2330,9 +2394,11 @@
     }
 
     protected void prepareCompactJobsForExternalDataset(List<Index> indexes, Dataset ds,
-            List<JobSpecification> jobsToExecute, MetadataProvider metadataProvider) throws AlgebricksException {
+            List<JobSpecification> jobsToExecute, MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws AlgebricksException {
         for (int j = 0; j < indexes.size(); j++) {
-            jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), metadataProvider));
+            jobsToExecute
+                    .add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), metadataProvider, sourceLoc));
 
         }
     }
@@ -2372,6 +2438,7 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             try {
                 final JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null);
+                afterCompile();
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
                 return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
@@ -2523,6 +2590,7 @@
 
     protected void handleCreateNodeGroupStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
+        SourceLocation sourceLoc = stmtCreateNodegroup.getSourceLocation();
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2532,7 +2600,8 @@
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
             if (ng != null) {
                 if (!stmtCreateNodegroup.getIfNotExists()) {
-                    throw new AlgebricksException("A nodegroup with this name " + ngName + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A nodegroup with this name " + ngName + " already exists.");
                 }
             } else {
                 List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
@@ -2554,6 +2623,7 @@
     protected void handleExternalDatasetRefreshStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
+        SourceLocation sourceLoc = stmtRefresh.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
         TransactionState transactionState = TransactionState.COMMIT;
@@ -2576,19 +2646,19 @@
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             // Dataset exists ?
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             // Dataset external ?
             if (ds.getDatasetType() != DatasetType.EXTERNAL) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "dataset " + datasetName + " in dataverse " + dataverseName + " is not an external dataset");
             }
             // Dataset has indexes ?
             indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
             if (indexes.isEmpty()) {
-                throw new AlgebricksException("External dataset " + datasetName + " in dataverse " + dataverseName
-                        + " doesn't have any index");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "External dataset " + datasetName
+                        + " in dataverse " + dataverseName + " doesn't have any index");
             }
 
             // Record transaction time
@@ -2648,7 +2718,7 @@
             for (Index index : indexes) {
                 if (!ExternalIndexingOperations.isFileIndex(index)) {
                     spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, addedFiles,
-                            appendedFiles, metadataProvider);
+                            appendedFiles, metadataProvider, sourceLoc);
                     // run the files update job
                     runJob(hcc, spec);
                 }
@@ -2780,6 +2850,11 @@
         return (dataverse != null) ? dataverse : activeDataverse.getDataverseName();
     }
 
+    @Override
+    public ExecutionPlans getExecutionPlans() {
+        return apiFramework.getExecutionPlans();
+    }
+
     public String getActiveDataverse(Identifier dataverse) {
         return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
     }
@@ -2815,9 +2890,16 @@
         rewriter.rewrite(stmt);
     }
 
-    private void ensureNonPrimaryIndexDrop(Index index) throws AlgebricksException {
+    private void ensureNonPrimaryIndexDrop(Index index, SourceLocation sourceLoc) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
-            throw new MetadataException(ErrorCode.CANNOT_DROP_INDEX, index.getIndexName(), index.getDatasetName());
+            throw new MetadataException(ErrorCode.CANNOT_DROP_INDEX, sourceLoc, index.getIndexName(),
+                    index.getDatasetName());
+        }
+    }
+
+    protected void afterCompile() {
+        if (sessionOutput.config().is(SessionConfig.FORMAT_HTML)) {
+            ExecutionPlansHtmlPrintUtil.print(sessionOutput.out(), getExecutionPlans());
         }
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
index adb3c18..c76d9b8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
@@ -24,14 +24,21 @@
 import java.util.Properties;
 
 import org.apache.asterix.common.config.AsterixProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.api.config.IConfigManager;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.control.common.controllers.ControllerConfig;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.common.utils.ConfigurationUtil;
 import org.apache.hyracks.util.file.FileUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 class ApplicationConfigurator {
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private ApplicationConfigurator() {
     }
 
@@ -56,4 +63,24 @@
         }
 
     }
+
+    public static void validateJavaRuntime() throws HyracksDataException {
+        final String javaVersion = System.getProperty("java.version");
+        LOGGER.info("Found JRE version " + javaVersion);
+        String[] splits = javaVersion.split("\\.");
+        if ("1".equals(splits[0])) {
+            switch (splits[1]) {
+                case "9":
+                    LOGGER.warn("JRE version \"" + javaVersion + "\" is untested");
+                    //fall-through
+                case "8":
+                    return;
+                default:
+                    throw RuntimeDataException.create(ErrorCode.UNSUPPORTED_JRE,
+                            "a minimum version of JRE of 1.8 is required, but is currently: \"" + javaVersion + "\"");
+            }
+        } else {
+            LOGGER.warn("JRE version \"" + javaVersion + "\" is untested");
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 7224526..b180ef0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -39,11 +39,8 @@
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
-import org.apache.asterix.api.http.server.DdlApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
-import org.apache.asterix.api.http.server.FullApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
-import org.apache.asterix.api.http.server.QueryApiServlet;
 import org.apache.asterix.api.http.server.QueryCancellationServlet;
 import org.apache.asterix.api.http.server.QueryResultApiServlet;
 import org.apache.asterix.api.http.server.QueryServiceServlet;
@@ -51,7 +48,6 @@
 import org.apache.asterix.api.http.server.RebalanceApiServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.ShutdownApiServlet;
-import org.apache.asterix.api.http.server.UpdateApiServlet;
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
@@ -59,7 +55,9 @@
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.MetadataProperties;
@@ -88,6 +86,7 @@
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.config.IConfigManager;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.control.cc.BaseCCApplication;
@@ -119,6 +118,7 @@
         ccServiceCtx = (ICCServiceContext) serviceCtx;
         ccServiceCtx.setThreadFactory(
                 new AsterixThreadFactory(ccServiceCtx.getThreadFactory(), new LifeCycleComponentManager()));
+        validateEnvironment();
     }
 
     @Override
@@ -144,12 +144,13 @@
         INcLifecycleCoordinator lifecycleCoordinator = createNcLifeCycleCoordinator(repProp.isReplicationEnabled());
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
-        GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
+
+        List<AsterixExtension> extensions = new ArrayList<>();
+        extensions.addAll(getExtensions());
+        ccExtensionManager = new CCExtensionManager(extensions);
+        IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         statementExecutorCtx = new StatementExecutorContext();
         appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator);
-        List<AsterixExtension> extensions = new ArrayList<>();
-        extensions.addAll(this.getExtensions());
-        ccExtensionManager = new CCExtensionManager(extensions);
         appCtx.setExtensionManager(ccExtensionManager);
         final CCConfig ccConfig = controllerService.getCCConfig();
         if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -175,15 +176,15 @@
     }
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
-            GlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
+            IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
             throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
                 globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
                 new MetadataLockManager());
     }
 
-    protected GlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
-        return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
+    protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
+        return ccExtensionManager.getGlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
     }
 
     protected INcLifecycleCoordinator createNcLifeCycleCoordinator(boolean replicationEnabled) {
@@ -196,8 +197,8 @@
         LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
     }
 
-    protected List<AsterixExtension> getExtensions() {
-        return appCtx.getExtensionProperties().getExtensions();
+    protected List<AsterixExtension> getExtensions() throws Exception {
+        return new ExtensionProperties(PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig())).getExtensions();
     }
 
     protected void configureServers() throws Exception {
@@ -235,18 +236,6 @@
         jsonAPIServer.setAttribute(ServletConstants.RUNNING_QUERIES_ATTR, statementExecutorCtx);
         jsonAPIServer.setAttribute(ServletConstants.SERVICE_CONTEXT_ATTR, ccServiceCtx);
 
-        // AQL rest APIs.
-        addServlet(jsonAPIServer, Servlets.AQL_QUERY);
-        addServlet(jsonAPIServer, Servlets.AQL_UPDATE);
-        addServlet(jsonAPIServer, Servlets.AQL_DDL);
-        addServlet(jsonAPIServer, Servlets.AQL);
-
-        // SQL+x+ rest APIs.
-        addServlet(jsonAPIServer, Servlets.SQLPP_QUERY);
-        addServlet(jsonAPIServer, Servlets.SQLPP_UPDATE);
-        addServlet(jsonAPIServer, Servlets.SQLPP_DDL);
-        addServlet(jsonAPIServer, Servlets.SQLPP);
-
         // Other APIs.
         addServlet(jsonAPIServer, Servlets.QUERY_STATUS);
         addServlet(jsonAPIServer, Servlets.QUERY_RESULT);
@@ -281,30 +270,6 @@
 
     protected IServlet createServlet(ConcurrentMap<String, Object> ctx, String key, String... paths) {
         switch (key) {
-            case Servlets.AQL:
-                return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_QUERY:
-                return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_UPDATE:
-                return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_DDL:
-                return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP:
-                return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_QUERY:
-                return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_UPDATE:
-                return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_DDL:
-                return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
             case Servlets.RUNNING_REQUESTS:
                 return new QueryCancellationServlet(ctx, paths);
             case Servlets.QUERY_STATUS:
@@ -372,4 +337,13 @@
     public IHyracksClientConnection getHcc() {
         return hcc;
     }
+
+    protected void validateEnvironment() throws HyracksDataException {
+        validateJavaVersion();
+    }
+
+    protected void validateJavaVersion() throws HyracksDataException {
+        ApplicationConfigurator.validateJavaRuntime();
+    }
+
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 3d9b822..5fc5c57 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -56,7 +56,7 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final IStorageComponentProvider componentProvider;
     protected final ICCServiceContext serviceCtx;
-    protected IHyracksClientConnection hcc;
+    protected final IHyracksClientConnection hcc;
     protected volatile boolean recoveryCompleted;
     protected volatile boolean recovering;
 
@@ -126,7 +126,9 @@
             throws Exception {
         // Loop over datasets
         for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
-            mdTxnCtx = recoverDataset(appCtx, mdTxnCtx, dataverse);
+            mdTxnCtx = recoverDatasets(appCtx, mdTxnCtx, dataverse);
+            // Fixes ASTERIXDB-2386 by caching the dataverse during recovery
+            MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse.getDataverseName());
         }
         return mdTxnCtx;
     }
@@ -138,8 +140,8 @@
         }
     }
 
-    private MetadataTransactionContext recoverDataset(ICcApplicationContext appCtx, MetadataTransactionContext mdTxnCtx,
-            Dataverse dataverse) throws Exception {
+    private MetadataTransactionContext recoverDatasets(ICcApplicationContext appCtx,
+            MetadataTransactionContext mdTxnCtx, Dataverse dataverse) throws Exception {
         if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
             MetadataProvider metadataProvider = new MetadataProvider(appCtx, dataverse);
             try {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 494198b..01bdd62 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.StorageApiServlet;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.nc.RecoveryManager;
 import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
@@ -44,6 +45,7 @@
 import org.apache.asterix.common.transactions.Checkpoint;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.utils.PrintUtil;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.common.utils.StorageConstants;
@@ -93,6 +95,7 @@
         ((NodeControllerService) serviceCtx.getControllerService()).setNodeStatus(NodeStatus.IDLE);
         ncServiceCtx.setThreadFactory(
                 new AsterixThreadFactory(ncServiceCtx.getThreadFactory(), ncServiceCtx.getLifeCycleComponentManager()));
+        validateEnvironment();
     }
 
     @Override
@@ -121,7 +124,7 @@
             }
             updateOnNodeJoin();
         }
-        runtimeContext.initialize(runtimeContext.getNodeProperties().isInitialRun());
+        runtimeContext.initialize(getRecoveryManagerFactory(), runtimeContext.getNodeProperties().isInitialRun());
         MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
         IMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
         this.ncServiceCtx.setMessageBroker(messageBroker);
@@ -144,6 +147,10 @@
         performLocalCleanUp();
     }
 
+    protected IRecoveryManagerFactory getRecoveryManagerFactory() {
+        return RecoveryManager::new;
+    }
+
     @Override
     protected void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
@@ -269,4 +276,13 @@
         }
         return state;
     }
+
+    protected void validateEnvironment() throws HyracksDataException {
+        validateJavaRuntime();
+    }
+
+    protected void validateJavaRuntime() throws HyracksDataException {
+        ApplicationConfigurator.validateJavaRuntime();
+    }
+
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 702b6b4..b74d739 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -355,7 +355,7 @@
         List<Index> indexes = metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         for (Index index : indexes) {
             jobs.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset,
-                    EnumSet.of(DropOption.IF_EXISTS, DropOption.WAIT_ON_IN_USE)));
+                    EnumSet.of(DropOption.IF_EXISTS, DropOption.WAIT_ON_IN_USE), null));
         }
         for (JobSpecification jobSpec : jobs) {
             JobUtils.runJob(hcc, jobSpec, true);
@@ -371,12 +371,12 @@
             }
             // Creates the secondary index.
             JobSpecification indexCreationJobSpec =
-                    IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider);
+                    IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider, null);
             JobUtils.runJob(hcc, indexCreationJobSpec, true);
 
             // Loads the secondary index.
             JobSpecification indexLoadingJobSpec =
-                    IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index, metadataProvider);
+                    IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index, metadataProvider, null);
             JobUtils.runJob(hcc, indexLoadingJobSpec, true);
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index acc3970..4b245a8 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -30,6 +30,7 @@
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BiPredicate;
 import java.util.stream.Stream;
 
@@ -70,6 +71,7 @@
             joinPath(getProjectPath().toString(), "src", "test", "resources", "cc.conf");
     private static final String DEFAULT_STORAGE_PATH = joinPath("target", "io", "dir");
     private static String storagePath = DEFAULT_STORAGE_PATH;
+    private static final long RESULT_TTL = TimeUnit.MINUTES.toMillis(5);
 
     static {
         System.setProperty("java.util.logging.manager", org.apache.logging.log4j.jul.LogManager.class.getName());
@@ -95,7 +97,8 @@
      * main method to run a simple 2 node cluster in-process
      * suggested VM arguments: <code>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8</code>
      *
-     * @param args unused
+     * @param args
+     *            unused
      */
     public static void main(String[] args) throws Exception {
         TestUtils.redirectLoggingToConsole();
@@ -198,7 +201,7 @@
         ccConfig.setClientListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ccConfig.setClientListenPort(DEFAULT_HYRACKS_CC_CLIENT_PORT);
         ccConfig.setClusterListenPort(DEFAULT_HYRACKS_CC_CLUSTER_PORT);
-        ccConfig.setResultTTL(120000L);
+        ccConfig.setResultTTL(RESULT_TTL);
         ccConfig.setResultSweepThreshold(1000L);
         ccConfig.setEnforceFrameWriterProtocol(true);
         configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb"));
@@ -217,18 +220,21 @@
         ncConfig.setDataListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ncConfig.setResultListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ncConfig.setMessagingListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
-        ncConfig.setResultTTL(120000L);
+        ncConfig.setResultTTL(RESULT_TTL);
         ncConfig.setResultSweepThreshold(1000L);
         ncConfig.setVirtualNC();
         configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb", ncName));
         return ncConfig;
     }
 
-    protected INCApplication createNCApplication() {
+    protected INCApplication createNCApplication()
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        // Instead of using this flag, RecoveryManagerTest should set the desired class in its config file
         if (!gracefulShutdown) {
             return new UngracefulShutdownNCApplication();
         }
-        return new NCApplication();
+        String ncAppClass = (String) configManager.get(NCConfig.Option.APP_CLASS);
+        return (INCApplication) Class.forName(ncAppClass).newInstance();
     }
 
     private NCConfig fixupIODevices(NCConfig ncConfig) throws IOException, AsterixException, CmdLineException {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 5cda9f2..c1ccda7 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -37,7 +37,6 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -58,17 +57,18 @@
 import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
 import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.runtime.CommitRuntime;
 import org.apache.asterix.transaction.management.service.logging.LogReader;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -166,7 +166,7 @@
 
     public void deInit(boolean cleanupOnStop) throws Exception {
         ExternalUDFLibrarian.removeLibraryDir();
-        ExecutionTestUtil.tearDown(cleanupOnStop);
+        ExecutionTestUtil.tearDown(cleanupOnStop, runHDFS);
     }
 
     public void setOpts(List<Pair<IOption, Object>> opts) {
@@ -186,7 +186,7 @@
         return new TxnId(jobId.getId());
     }
 
-    public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+    public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
             Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
             int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
             StorageComponentProvider storageComponentProvider, Index secondaryIndex)
@@ -195,7 +195,39 @@
                 primaryKeyIndicators, storageComponentProvider, secondaryIndex, IndexOperation.INSERT);
     }
 
-    public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+    public Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> getBulkLoadSecondaryOperator(
+            IHyracksTaskContext ctx, Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType,
+            ARecordType metaType, int[] filterFields, int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
+            StorageComponentProvider storageComponentProvider, Index secondaryIndex, int numElementsHint)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        try {
+            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            org.apache.hyracks.algebricks.common.utils.Pair<ILSMMergePolicyFactory, Map<String, String>> mergePolicy =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
+                    mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
+            SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
+            IIndexDataflowHelperFactory secondaryIndexHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), secondaryIndexInfo.fileSplitProvider);
+            IIndexDataflowHelperFactory primaryIndexHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+            int[] fieldPermutation = new int[secondaryIndex.getKeyFieldNames().size()];
+            for (int i = 0; i < fieldPermutation.length; i++) {
+                fieldPermutation[i] = i;
+            }
+            LSMIndexBulkLoadOperatorNodePushable op =
+                    new LSMIndexBulkLoadOperatorNodePushable(secondaryIndexHelperFactory, primaryIndexHelperFactory,
+                            ctx, 0, fieldPermutation, 1.0F, false, numElementsHint, true, secondaryIndexInfo.rDesc,
+                            BulkLoadUsage.CREATE_INDEX, dataset.getDatasetId());
+            op.setOutputFrameWriter(0, new SinkRuntimeFactory().createPushRuntime(ctx)[0], null);
+            return Pair.of(secondaryIndexInfo, op);
+        } catch (Throwable th) {
+            throw HyracksDataException.create(th);
+        }
+    }
+
+    public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
             Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
             int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
             StorageComponentProvider storageComponentProvider, Index secondaryIndex, IndexOperation op)
@@ -210,10 +242,8 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
                     mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
-            IModificationOperationCallbackFactory modOpCallbackFactory =
-                    new PrimaryIndexModificationOperationCallbackFactory(dataset.getDatasetId(),
-                            primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, Operation.get(op),
-                            ResourceType.LSM_BTREE);
+            IModificationOperationCallbackFactory modOpCallbackFactory = dataset.getModificationCallbackFactory(
+                    storageComponentProvider, primaryIndexInfo.index, op, primaryIndexInfo.primaryKeyIndexes);
             IRecordDescriptorProvider recordDescProvider = primaryIndexInfo.getInsertRecordDescriptorProvider();
             RecordDescriptor recordDesc =
                     recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0), 0);
@@ -222,7 +252,7 @@
             LSMInsertDeleteOperatorNodePushable insertOp =
                     new LSMInsertDeleteOperatorNodePushable(ctx, ctx.getTaskAttemptId().getTaskId().getPartition(),
                             primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDesc, op, true,
-                            indexHelperFactory, modOpCallbackFactory, null);
+                            indexHelperFactory, modOpCallbackFactory, null, null);
 
             // For now, this assumes a single secondary index. recordDesc is always <pk-record-meta>
             // for the index, we will have to create an assign operator that extract the sk
@@ -237,8 +267,9 @@
                             ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
                             : recordType;
                     int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
-                    secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            mdProvider.getFunctionManager(), sourceType, secondaryIndex.getKeyFieldNames().get(i), pos);
+                    secondaryFieldAccessEvalFactories[i] =
+                            mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
+                                    sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
                 }
                 // outColumns are computed inside the assign runtime
                 int[] outColumns = new int[skNames.size()];
@@ -270,18 +301,20 @@
                 LSMInsertDeleteOperatorNodePushable secondaryInsertOp =
                         new LSMInsertDeleteOperatorNodePushable(ctx, ctx.getTaskAttemptId().getTaskId().getPartition(),
                                 secondaryIndexInfo.insertFieldsPermutations, secondaryIndexInfo.rDesc, op, false,
-                                secondaryIndexHelperFactory, secondaryModCallbackFactory, null);
+                                secondaryIndexHelperFactory, secondaryModCallbackFactory, null, null);
                 assignOp.setOutputFrameWriter(0, secondaryInsertOp, secondaryIndexInfo.rDesc);
-                CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
-                        secondaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
-                        true);
+
+                IPushRuntime commitOp =
+                        dataset.getCommitRuntimeFactory(mdProvider, secondaryIndexInfo.primaryKeyIndexes, true)
+                                .createPushRuntime(ctx)[0];
+
                 secondaryInsertOp.setOutputFrameWriter(0, commitOp, secondaryIndexInfo.rDesc);
                 commitOp.setInputRecordDescriptor(0, secondaryIndexInfo.rDesc);
                 return Pair.of(insertOp, commitOp);
             } else {
-                CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
-                        primaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
-                        true);
+                IPushRuntime commitOp =
+                        dataset.getCommitRuntimeFactory(mdProvider, primaryIndexInfo.primaryKeyIndexes, true)
+                                .createPushRuntime(ctx)[0];
                 insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
                 commitOp.setInputRecordDescriptor(0, primaryIndexInfo.rDesc);
                 return Pair.of(insertOp, commitOp);
@@ -380,7 +413,6 @@
                 (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), dataverse);
         SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
         try {
-
             IResourceFactory resourceFactory = primaryIndexInfo.dataset.getResourceFactory(mdProvider, secondaryIndex,
                     primaryIndexInfo.recordType, primaryIndexInfo.metaType, mergePolicy.first, mergePolicy.second);
             IndexBuilderFactory indexBuilderFactory =
@@ -419,9 +451,9 @@
             secondaryIndexSerdes[i] =
                     SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(secondaryKeyTypes[i]);
         }
-        for (; i < primaryKeyTypes.length; i++) {
-            secondaryIndexSerdes[i] =
-                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
+        for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+            secondaryIndexSerdes[i] = SerializerDeserializerProvider.INSTANCE
+                    .getSerializerDeserializer(primaryKeyTypes[i - secondaryKeyTypes.length]);
         }
         return secondaryIndexSerdes;
     }
@@ -447,8 +479,9 @@
         for (; i < secondaryKeyTypes.length; i++) {
             secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(secondaryKeyTypes[i]);
         }
-        for (; i < primaryKeyTypes.length; i++) {
-            secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i]);
+        for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+            secondaryIndexTypeTraits[i] =
+                    TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i - secondaryKeyTypes.length]);
         }
         return secondaryIndexTypeTraits;
     }
@@ -491,12 +524,14 @@
     }
 
     public static class SecondaryIndexInfo {
-        private final int[] primaryKeyIndexes;
-        private final PrimaryIndexInfo primaryIndexInfo;
-        private final Index secondaryIndex;
-        private final ConstantFileSplitProvider fileSplitProvider;
-        private final RecordDescriptor rDesc;
-        private final int[] insertFieldsPermutations;
+        final int[] primaryKeyIndexes;
+        final PrimaryIndexInfo primaryIndexInfo;
+        final Index secondaryIndex;
+        final ConstantFileSplitProvider fileSplitProvider;
+        final ISerializerDeserializer<?>[] secondaryIndexSerdes;
+        final RecordDescriptor rDesc;
+        final int[] insertFieldsPermutations;
+        final ITypeTraits[] secondaryIndexTypeTraits;
 
         public SecondaryIndexInfo(PrimaryIndexInfo primaryIndexInfo, Index secondaryIndex) {
             this.primaryIndexInfo = primaryIndexInfo;
@@ -507,11 +542,11 @@
             FileSplit[] splits = SplitsAndConstraintsUtil.getIndexSplits(appCtx.getClusterStateManager(),
                     primaryIndexInfo.dataset, secondaryIndex.getIndexName(), nodes);
             fileSplitProvider = new ConstantFileSplitProvider(splits);
-            ITypeTraits[] secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
+            secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
                     primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
                     secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
-            ISerializerDeserializer<?>[] secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType,
-                    primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
+            secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType, primaryIndexInfo.metaType,
+                    primaryIndexInfo.primaryKeyTypes,
                     secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
             rDesc = new RecordDescriptor(secondaryIndexSerdes, secondaryIndexTypeTraits);
             insertFieldsPermutations = new int[secondaryIndexTypeTraits.length];
@@ -527,6 +562,10 @@
         public IFileSplitProvider getFileSplitProvider() {
             return fileSplitProvider;
         }
+
+        public ISerializerDeserializer<?>[] getSerdes() {
+            return secondaryIndexSerdes;
+        }
     }
 
     public static class PrimaryIndexInfo {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
index 2eba473..ff027e2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class ABooleanFieldValueGenerator implements IAsterixFieldValueGenerator<Boolean> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
index e698676..64dab3d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class ADoubleFieldValueGenerator implements IAsterixFieldValueGenerator<Double> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
index 7c6556b..5540c11 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AInt32FieldValueGenerator implements IAsterixFieldValueGenerator<Integer> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
index 2a2496e..e62054a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AInt64FieldValueGenerator implements IAsterixFieldValueGenerator<Long> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
index cd4de62..b242189 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
@@ -21,7 +21,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
index 5ee6d40..419d1b6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.util.string.UTF8StringReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
new file mode 100644
index 0000000..1b04c1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
@@ -0,0 +1,126 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.asterix.app.data.gen;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.test.common.TestTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class RecordTupleGenerator {
+
+    private final int[] keyIndexes;
+    private final int[] keyIndicators;
+    private final ARecordValueGenerator recordGenerator;
+    private final ARecordValueGenerator metaGenerator;
+    private final TestTupleReference tuple;
+
+    public enum GenerationFunction {
+        RANDOM,
+        DETERMINISTIC,
+        INCREASING,
+        DECREASING
+    }
+
+    /**
+     * @param recordType
+     * @param metaType
+     * @param key
+     * @param keyIndexes
+     * @param keyIndicators
+     * @param recordGeneration
+     * @param uniqueRecordFields
+     * @param metaGeneration
+     * @param uniqueMetaFields
+     */
+    public RecordTupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
+            GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
+            boolean[] uniqueMetaFields) {
+        this.keyIndexes = keyIndexes;
+        this.keyIndicators = keyIndicators;
+        for (IAType field : recordType.getFieldTypes()) {
+            validate(field);
+        }
+        recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
+        if (metaType != null) {
+            for (IAType field : metaType.getFieldTypes()) {
+                validate(field);
+            }
+            metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
+        } else {
+            metaGenerator = null;
+        }
+        int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
+        tuple = new TestTupleReference(numOfFields);
+        boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
+        for (int i = 0; i < keyIndexes.length; i++) {
+            if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
+                throw new IllegalArgumentException("key field indicator must be either 0 or 1");
+            }
+            atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
+                    || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
+                            keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
+        }
+        if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
+            throw new IllegalArgumentException("at least one key field must be unique and not boolean");
+        }
+        if (keyIndexes.length != keyIndicators.length) {
+            throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
+        }
+    }
+
+    private boolean validateKey(int i, ARecordType type, boolean unique) {
+        if (type.getFieldNames().length <= i) {
+            throw new IllegalArgumentException("key index must be less than number of fields");
+        }
+        return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
+    }
+
+    public ITupleReference next() throws IOException {
+        tuple.reset();
+        recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
+        if (metaGenerator != null) {
+            recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
+        }
+        for (int i = 0; i < keyIndexes.length; i++) {
+            if (keyIndicators[i] == 0) {
+                recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+            } else {
+                metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+            }
+        }
+        return tuple;
+    }
+
+    private void validate(IAType field) {
+        switch (field.getTypeTag()) {
+            case BOOLEAN:
+            case DOUBLE:
+            case INTEGER:
+            case BIGINT:
+            case STRING:
+                break;
+            default:
+                throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
deleted file mode 100644
index 0469349..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-package org.apache.asterix.app.data.gen;
-
-import java.io.IOException;
-
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public class TupleGenerator {
-
-    private final int[] keyIndexes;
-    private final int[] keyIndicators;
-    private final ARecordValueGenerator recordGenerator;
-    private final ARecordValueGenerator metaGenerator;
-    private final TestTupleReference tuple;
-
-    public enum GenerationFunction {
-        RANDOM,
-        DETERMINISTIC,
-        INCREASING,
-        DECREASING
-    }
-
-    /**
-     * @param recordType
-     * @param metaType
-     * @param key
-     * @param keyIndexes
-     * @param keyIndicators
-     * @param recordGeneration
-     * @param uniqueRecordFields
-     * @param metaGeneration
-     * @param uniqueMetaFields
-     */
-    public TupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
-            GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
-            boolean[] uniqueMetaFields) {
-        this.keyIndexes = keyIndexes;
-        this.keyIndicators = keyIndicators;
-        for (IAType field : recordType.getFieldTypes()) {
-            validate(field);
-        }
-        recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
-        if (metaType != null) {
-            for (IAType field : metaType.getFieldTypes()) {
-                validate(field);
-            }
-            metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
-        } else {
-            metaGenerator = null;
-        }
-        int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
-        tuple = new TestTupleReference(numOfFields);
-        boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
-        for (int i = 0; i < keyIndexes.length; i++) {
-            if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
-                throw new IllegalArgumentException("key field indicator must be either 0 or 1");
-            }
-            atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
-                    || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
-                            keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
-        }
-        if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
-            throw new IllegalArgumentException("at least one key field must be unique and not boolean");
-        }
-        if (keyIndexes.length != keyIndicators.length) {
-            throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
-        }
-    }
-
-    private boolean validateKey(int i, ARecordType type, boolean unique) {
-        if (type.getFieldNames().length <= i) {
-            throw new IllegalArgumentException("key index must be less than number of fields");
-        }
-        return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
-    }
-
-    public ITupleReference next() throws IOException {
-        tuple.reset();
-        recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
-        if (metaGenerator != null) {
-            recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
-        }
-        for (int i = 0; i < keyIndexes.length; i++) {
-            if (keyIndicators[i] == 0) {
-                recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
-            } else {
-                metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
-            }
-        }
-        return tuple;
-    }
-
-    private void validate(IAType field) {
-        switch (field.getTypeTag()) {
-            case BOOLEAN:
-            case DOUBLE:
-            case INTEGER:
-            case BIGINT:
-            case STRING:
-                break;
-            default:
-                throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
index baaf546..ccbf4e3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
@@ -67,9 +67,9 @@
         while (argEvalFactoryIterator.hasNext()) {
             IScalarEvaluatorFactory evalFactory = funcDesc.createEvaluatorFactory(argEvalFactoryIterator.next());
             IHyracksTaskContext ctx = mock(IHyracksTaskContext.class);
-            IScalarEvaluator evaluator = evalFactory.createScalarEvaluator(ctx);
-            IPointable resultPointable = new VoidPointable();
             try {
+                IScalarEvaluator evaluator = evalFactory.createScalarEvaluator(ctx);
+                IPointable resultPointable = new VoidPointable();
                 evaluator.evaluate(null, resultPointable);
             } catch (Throwable e) {
                 String msg = e.getMessage();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index a1e70dc..942b192 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -100,7 +100,8 @@
     }
 
     @Override
-    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount) {
+    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount,
+            boolean expectedSourceLoc) {
         // Get the expected exception
         for (Iterator<String> iter = expectedErrors.iterator(); iter.hasNext();) {
             String expectedError = iter.next();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index a356d23..43833a2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -53,7 +53,8 @@
         SIGNATURE("signature"),
         STATUS("status"),
         TYPE("type"),
-        ERRORS("errors");
+        ERRORS("errors"),
+        PLANS("plans");
 
         private static final Map<String, ResultField> fields = new HashMap<>();
 
@@ -162,6 +163,7 @@
                 case SIGNATURE:
                 case STATUS:
                 case TYPE:
+                case PLANS:
                     resultBuilder.append(OBJECT_MAPPER.writeValueAsString(fieldValue));
                     break;
                 default:
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 80048bd..4697a2a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -109,6 +109,8 @@
      * Static variables
      */
     protected static final Logger LOGGER = LogManager.getLogger();
+    private static final String AQL = "aql";
+    private static final String SQLPP = "sqlpp";
     // see
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
@@ -561,14 +563,6 @@
         }
     }
 
-    public InputStream executeQuery(String str, OutputFormat fmt, URI uri, List<Parameter> params) throws Exception {
-        HttpUriRequest method = constructHttpMethod(str, uri, "query", false, params);
-        // Set accepted output response type
-        method.setHeader("Accept", fmt.mimeType());
-        HttpResponse response = executeAndCheckHttpRequest(method);
-        return response.getEntity().getContent();
-    }
-
     public InputStream executeQueryService(String str, URI uri, OutputFormat fmt) throws Exception {
         return executeQueryService(str, fmt, uri, new ArrayList<>(), false);
     }
@@ -591,6 +585,10 @@
             newParams = upsertParam(newParams, QueryServiceServlet.Parameter.MAX_RESULT_READS.str(),
                     maxReadsOptional.get());
         }
+        final List<Parameter> additionalParams = extractParameters(str);
+        for (Parameter param : additionalParams) {
+            newParams = upsertParam(newParams, param.getName(), param.getValue());
+        }
         HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", newParams)
                 : constructPostMethodUrl(str, uri, "statement", newParams);
         // Set accepted output response type
@@ -626,15 +624,9 @@
     }
 
     private HttpUriRequest constructHttpMethod(String statement, URI uri, String stmtParam, boolean postStmtAsParam,
-            List<Parameter> otherParams) throws URISyntaxException {
-        if (statement.length() + uri.toString().length() < MAX_URL_LENGTH) {
-            // Use GET for small-ish queries
-            return constructGetMethod(uri, upsertParam(otherParams, stmtParam, statement));
-        } else {
-            // Use POST for bigger ones to avoid 413 FULL_HEAD
-            String stmtParamName = (postStmtAsParam ? stmtParam : null);
-            return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
-        }
+            List<Parameter> otherParams) {
+        String stmtParamName = (postStmtAsParam ? stmtParam : null);
+        return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
     }
 
     private HttpUriRequest constructGetMethod(URI endpoint, List<Parameter> params) {
@@ -653,9 +645,7 @@
             builder.addParameter(param.getName(), param.getValue());
         }
         builder.setCharset(StandardCharsets.UTF_8);
-        if (body.isPresent()) {
-            builder.setEntity(new StringEntity(body.get(), StandardCharsets.UTF_8));
-        }
+        body.ifPresent(s -> builder.setEntity(new StringEntity(s, StandardCharsets.UTF_8)));
         return builder.build();
     }
 
@@ -683,13 +673,6 @@
         return builder.build();
     }
 
-    private HttpUriRequest constructPostMethod(URI uri, OutputFormat fmt, List<Parameter> params) {
-        HttpUriRequest method = constructPostMethod(uri, params);
-        // Set accepted output response type
-        method.setHeader("Accept", fmt.mimeType());
-        return method;
-    }
-
     protected HttpUriRequest constructPostMethodUrl(String statement, URI uri, String stmtParam,
             List<Parameter> otherParams) {
         RequestBuilder builder = RequestBuilder.post(uri);
@@ -751,57 +734,6 @@
         return response.getEntity().getContent();
     }
 
-    // To execute Update statements
-    // Insert and Delete statements are executed here
-    public void executeUpdate(String str, URI uri) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
-        // Execute the method.
-        executeAndCheckHttpRequest(request);
-    }
-
-    // Executes AQL in either async or async-defer mode.
-    public InputStream executeAnyAQLAsync(String statement, boolean defer, OutputFormat fmt, URI uri,
-            Map<String, Object> variableCtx) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).addParameter("mode", defer ? "asynchronous-deferred" : "asynchronous")
-                        .setEntity(new StringEntity(statement, StandardCharsets.UTF_8))
-                        .setHeader("Accept", fmt.mimeType()).build();
-
-        String handleVar = getHandleVariable(statement);
-
-        HttpResponse response = executeAndCheckHttpRequest(request);
-        InputStream resultStream = response.getEntity().getContent();
-        String resultStr = IOUtils.toString(resultStream, "UTF-8");
-        ObjectNode resultJson = new ObjectMapper().readValue(resultStr, ObjectNode.class);
-        final JsonNode jsonHandle = resultJson.get("handle");
-        final String strHandle = jsonHandle.asText();
-
-        if (handleVar != null) {
-            variableCtx.put(handleVar, strHandle);
-            return resultStream;
-        }
-        return null;
-    }
-
-    // To execute DDL and Update statements
-    // create type statement
-    // create dataset statement
-    // create index statement
-    // create dataverse statement
-    // create function statement
-    public void executeDDL(String str, URI uri) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
-        // Execute the method.
-        executeAndCheckHttpRequest(request);
-    }
-
     // Method that reads a DDL/Update/Query File
     // and returns the contents as a string
     // This string is later passed to REST API for execution.
@@ -886,13 +818,15 @@
             String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
             MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
             throws Exception {
+        URI uri;
+        InputStream resultStream;
         File qbcFile;
         boolean failed = false;
         File expectedResultFile;
         switch (ctx.getType()) {
             case "ddl":
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
                 } else {
                     executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
                 }
@@ -903,7 +837,7 @@
                     statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
                 }
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } else {
                     executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 }
@@ -957,14 +891,12 @@
                         expectedResultFileCtxs);
                 break;
             case "txnqbc": // qbc represents query before crash
-                InputStream resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                resultStream = query(cUnit, testFile.getName(), statement);
                 qbcFile = getTestCaseQueryBeforeCrashFile(actualPath, testCaseCtx, cUnit);
                 writeOutputToFile(qbcFile, resultStream);
                 break;
             case "txnqar": // qar represents query after recovery
-                resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                resultStream = query(cUnit, testFile.getName(), statement);
                 File qarFile = new File(actualPath + File.separator
                         + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_" + cUnit.getName()
                         + "_qar.adm");
@@ -974,7 +906,7 @@
                 break;
             case "txneu": // eu represents erroneous update
                 try {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } catch (Exception e) {
                     // An exception is expected.
                     failed = true;
@@ -1001,7 +933,7 @@
                 break;
             case "errddl": // a ddlquery that expects error
                 try {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } catch (Exception e) {
                     // expected error happens
                     failed = true;
@@ -1238,38 +1170,25 @@
     public void executeQuery(OutputFormat fmt, String statement, Map<String, Object> variableCtx, String reqType,
             File testFile, File expectedResultFile, File actualResultFile, MutableInt queryCount, int numResultFiles,
             List<Parameter> params, ComparisonEnum compare) throws Exception {
-        InputStream resultStream = null;
-        if (testFile.getName().endsWith("aql")) {
-            if (reqType.equalsIgnoreCase("query")) {
-                resultStream = executeQuery(statement, fmt, getEndpoint(Servlets.AQL_QUERY), params);
-            } else {
-                final URI endpoint = getEndpoint(Servlets.AQL);
-                if (reqType.equalsIgnoreCase("async")) {
-                    resultStream = executeAnyAQLAsync(statement, false, fmt, endpoint, variableCtx);
-                } else if (reqType.equalsIgnoreCase("deferred")) {
-                    resultStream = executeAnyAQLAsync(statement, true, fmt, endpoint, variableCtx);
-                }
-                Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), resultStream);
-            }
+        String delivery = DELIVERY_IMMEDIATE;
+        if (reqType.equalsIgnoreCase("async")) {
+            delivery = DELIVERY_ASYNC;
+        } else if (reqType.equalsIgnoreCase("deferred")) {
+            delivery = DELIVERY_DEFERRED;
+        }
+        URI uri = testFile.getName().endsWith("aql") ? getEndpoint(Servlets.QUERY_AQL)
+                : getEndpoint(Servlets.QUERY_SERVICE);
+        InputStream resultStream;
+        if (DELIVERY_IMMEDIATE.equals(delivery)) {
+            resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
+            resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
+                    : ResultExtractor.extract(resultStream);
         } else {
-            String delivery = DELIVERY_IMMEDIATE;
-            if (reqType.equalsIgnoreCase("async")) {
-                delivery = DELIVERY_ASYNC;
-            } else if (reqType.equalsIgnoreCase("deferred")) {
-                delivery = DELIVERY_DEFERRED;
-            }
-            final URI uri = getEndpoint(Servlets.QUERY_SERVICE);
-            if (DELIVERY_IMMEDIATE.equals(delivery)) {
-                resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
-                resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
-                        : ResultExtractor.extract(resultStream);
-            } else {
-                String handleVar = getHandleVariable(statement);
-                resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
-                String handle = ResultExtractor.extractHandle(resultStream);
-                Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
-                variableCtx.put(handleVar, handle);
-            }
+            String handleVar = getHandleVariable(statement);
+            resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
+            String handle = ResultExtractor.extractHandle(resultStream);
+            Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
+            variableCtx.put(handleVar, toQueryServiceHandle(handle));
         }
         if (actualResultFile == null) {
             if (testFile.getName().startsWith(DIAGNOSE)) {
@@ -1440,7 +1359,16 @@
     }
 
     public InputStream executeSqlppUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
-        InputStream resultStream = executeQueryService(statement, getEndpoint(Servlets.QUERY_SERVICE), outputFormat);
+        return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP));
+    }
+
+    private InputStream executeAqlUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
+        return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(AQL));
+    }
+
+    private InputStream executeUpdateOrDdl(String statement, OutputFormat outputFormat, URI serviceUri)
+            throws Exception {
+        InputStream resultStream = executeQueryService(statement, serviceUri, outputFormat);
         return ResultExtractor.extract(resultStream);
     }
 
@@ -1641,7 +1569,8 @@
                     }
                 } catch (Exception e) {
                     numOfErrors++;
-                    boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount);
+                    boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount,
+                            testCaseCtx.isSourceLocationExpected(cUnit));
                     if (unexpected) {
                         LOGGER.error("testFile {} raised an unexpected exception", testFile, e);
                         if (failedGroup != null) {
@@ -1687,22 +1616,41 @@
         throw new Exception("Test \"" + testFile + "\" FAILED!", e);
     }
 
-    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount) {
+    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount,
+            boolean expectedSourceLoc) {
         String expectedError = null;
         if (expectedErrors.size() < numOfErrors) {
             return true;
         } else {
             // Get the expected exception
             expectedError = expectedErrors.get(numOfErrors - 1);
-            if (e.toString().contains(expectedError)) {
-                return false;
-            } else {
+            String actualError = e.toString();
+            if (!actualError.contains(expectedError)) {
                 LOGGER.error("Expected to find the following in error text: +++++{}+++++", expectedError);
                 return true;
             }
+            if (expectedSourceLoc && !containsSourceLocation(actualError)) {
+                LOGGER.error("Expected to find source location \"{}, {}\" in error text: +++++{}+++++",
+                        ERR_MSG_SRC_LOC_LINE_REGEX, ERR_MSG_SRC_LOC_COLUMN_REGEX, actualError);
+                return true;
+            }
+            return false;
         }
     }
 
+    private static final String ERR_MSG_SRC_LOC_LINE_REGEX = "in line \\d+";
+    private static final Pattern ERR_MSG_SRC_LOC_LINE_PATTERN =
+            Pattern.compile(ERR_MSG_SRC_LOC_LINE_REGEX, Pattern.CASE_INSENSITIVE);
+
+    private static final String ERR_MSG_SRC_LOC_COLUMN_REGEX = "at column \\d+";
+    private static final Pattern ERR_MSG_SRC_LOC_COLUMN_PATTERN =
+            Pattern.compile(ERR_MSG_SRC_LOC_COLUMN_REGEX, Pattern.CASE_INSENSITIVE);
+
+    private boolean containsSourceLocation(String errorMessage) {
+        Matcher lineMatcher = ERR_MSG_SRC_LOC_LINE_PATTERN.matcher(errorMessage);
+        return lineMatcher.find() && ERR_MSG_SRC_LOC_COLUMN_PATTERN.matcher(errorMessage).find(lineMatcher.end());
+    }
+
     private static File getTestCaseQueryBeforeCrashFile(String actualPath, TestCaseContext testCaseCtx,
             CompilationUnit cUnit) {
         return new File(
@@ -1947,4 +1895,19 @@
     private static boolean isCancellable(String type) {
         return !NON_CANCELLABLE.contains(type);
     }
+
+    private InputStream query(CompilationUnit cUnit, String testFile, String statement) throws Exception {
+        final URI uri = getQueryServiceUri(testFile);
+        final InputStream inputStream = executeQueryService(statement, OutputFormat.forCompilationUnit(cUnit), uri,
+                cUnit.getParameter(), true, null, false);
+        return ResultExtractor.extract(inputStream);
+    }
+
+    private URI getQueryServiceUri(String extension) throws URISyntaxException {
+        return extension.endsWith(AQL) ? getEndpoint(Servlets.QUERY_AQL) : getEndpoint(Servlets.QUERY_SERVICE);
+    }
+
+    private static String toQueryServiceHandle(String handle) {
+        return handle.replace("/aql/", "/service/");
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
new file mode 100644
index 0000000..c34f5a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt32;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt32ValueGenerator implements IFieldValueGenerator<AInt32> {
+    int counter = 0;
+
+    @Override
+    public AInt32 next() {
+        return new AInt32(counter++);
+    }
+
+    @Override
+    public void reset() {
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
new file mode 100644
index 0000000..b860737
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt64;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt64ValueGenerator implements IFieldValueGenerator<AInt64> {
+    long counter = 0L;
+
+    @Override
+    public AInt64 next() {
+        return new AInt64(counter++);
+    }
+
+    @Override
+    public void reset() {
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
new file mode 100644
index 0000000..a0ed26e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -0,0 +1,577 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.dataflow;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.app.bootstrap.TestNodeController;
+import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
+import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.IndexCheckpoint;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.TransactionOptions;
+import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
+import org.apache.asterix.test.common.TestHelper;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class CheckpointInSecondaryIndexTest {
+    static final int REPREAT_TEST_COUNT = 1;
+
+    @Parameterized.Parameters
+    public static List<Object[]> data() {
+        return Arrays.asList(new Object[REPREAT_TEST_COUNT][0]);
+    }
+
+    private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
+    private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
+            new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
+    private static final GenerationFunction[] RECORD_GEN_FUNCTION =
+            { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
+    private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
+    private static final ARecordType META_TYPE = null;
+    private static final GenerationFunction[] META_GEN_FUNCTION = null;
+    private static final boolean[] UNIQUE_META_FIELDS = null;
+    private static final int[] KEY_INDEXES = { 0 };
+    private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
+    private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
+    private static final int RECORDS_PER_COMPONENT = 500;
+    private static final int DATASET_ID = 101;
+    private static final String DATAVERSE_NAME = "TestDV";
+    private static final String DATASET_NAME = "TestDS";
+    private static final String INDEX_NAME = "TestIdx";
+    private static final String DATA_TYPE_NAME = "DUMMY";
+    private static final String NODE_GROUP_NAME = "DEFAULT";
+    private static final IndexType INDEX_TYPE = IndexType.BTREE;
+    private static final IFieldValueGenerator[] SECONDARY_INDEX_VALUE_GENERATOR =
+            { new AInt64ValueGenerator(), new AInt32ValueGenerator() };
+    private static final List<List<String>> INDEX_FIELD_NAMES =
+            Arrays.asList(Arrays.asList(RECORD_TYPE.getFieldNames()[1]));
+    private static final List<Integer> INDEX_FIELD_INDICATORS = Arrays.asList(Index.RECORD_INDICATOR);
+    private static final List<IAType> INDEX_FIELD_TYPES = Arrays.asList(BuiltinType.AINT64);
+    private static final StorageComponentProvider storageManager = new StorageComponentProvider();
+    private static TestNodeController nc;
+    private static NCAppRuntimeContext ncAppCtx;
+    private static IDatasetLifecycleManager dsLifecycleMgr;
+    private static Dataset dataset;
+    private static Index secondaryIndex;
+    private static ITransactionContext txnCtx;
+    private static TestLsmBtree primaryLsmBtree;
+    private static TestLsmBtree secondaryLsmBtree;
+    private static PrimaryIndexInfo primaryIndexInfo;
+    private static IHyracksTaskContext taskCtx;
+    private static IIndexDataflowHelper primaryIndexDataflowHelper;
+    private static IIndexDataflowHelper secondaryIndexDataflowHelper;
+    private static LSMInsertDeleteOperatorNodePushable insertOp;
+    private static LSMIndexBulkLoadOperatorNodePushable indexLoadOp;
+    private static IHyracksTaskContext loadTaskCtx;
+    private static SecondaryIndexInfo secondaryIndexInfo;
+    private static Actor actor;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        System.out.println("SetUp: ");
+        TestHelper.deleteExistingInstanceFiles();
+        String configPath = System.getProperty("user.dir") + File.separator + "src" + File.separator + "test"
+                + File.separator + "resources" + File.separator + "cc-multipart.conf";
+        nc = new TestNodeController(configPath, false);
+        nc.init();
+        ncAppCtx = nc.getAppRuntimeContext();
+        dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        System.out.println("TearDown");
+        nc.deInit();
+        TestHelper.deleteExistingInstanceFiles();
+    }
+
+    @Before
+    public void createIndex() throws Exception {
+        List<List<String>> partitioningKeys = new ArrayList<>();
+        partitioningKeys.add(Collections.singletonList("key"));
+        dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME,
+                NODE_GROUP_NAME, NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null,
+                        PartitioningStrategy.HASH, partitioningKeys, null, null, null, false, null),
+                null, DatasetType.INTERNAL, DATASET_ID, 0);
+        secondaryIndex = new Index(DATAVERSE_NAME, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
+                INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0);
+        taskCtx = null;
+        primaryIndexDataflowHelper = null;
+        secondaryIndexDataflowHelper = null;
+        primaryLsmBtree = null;
+        insertOp = null;
+        JobId jobId = nc.newJobId();
+        txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(jobId),
+                new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
+        actor = null;
+        taskCtx = nc.createTestContext(jobId, 0, false);
+        primaryIndexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+                KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+        IndexDataflowHelperFactory iHelperFactory =
+                new IndexDataflowHelperFactory(nc.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+        primaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+        primaryIndexDataflowHelper.open();
+        primaryLsmBtree = (TestLsmBtree) primaryIndexDataflowHelper.getIndexInstance();
+        primaryIndexDataflowHelper.close();
+        // This pipeline skips the secondary index
+        insertOp = nc.getInsertPipeline(taskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, storageManager, null).getLeft();
+        actor = new Actor("player");
+        // allow all operations
+        StorageTestUtils.allowAllOps(primaryLsmBtree);
+        actor.add(new Request(Request.Action.INSERT_OPEN));
+    }
+
+    @After
+    public void destroyIndex() throws Exception {
+        Request close = new Request(Request.Action.INSERT_CLOSE);
+        actor.add(close);
+        close.await();
+        nc.getTransactionManager().commitTransaction(txnCtx.getTxnId());
+        if (secondaryIndexDataflowHelper != null) {
+            secondaryIndexDataflowHelper.destroy();
+        }
+        primaryIndexDataflowHelper.destroy();
+        actor.stop();
+    }
+
+    @Test
+    public void testCheckpointUpdatedWhenSecondaryIsEmpty() throws Exception {
+        try {
+            // create secondary
+            createSecondaryIndex();
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // and that secondary index is empty
+            Assert.assertTrue(secondaryLsmBtree.isCurrentMutableComponentEmpty());
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+
+            // ensure secondary doesn't have a component
+            Assert.assertEquals(0, secondaryLsmBtree.getDiskComponents().size());
+            // ensure that current memory component index match
+            Assert.assertEquals(secondaryLsmBtree.getCurrentMemoryComponentIndex(),
+                    primaryLsmBtree.getCurrentMemoryComponentIndex());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            // secondary ref
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    private void createSecondaryIndex()
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        SecondaryIndexInfo secondaryIndexInfo =
+                nc.createSecondaryIndex(primaryIndexInfo, secondaryIndex, storageManager, 0);
+        IndexDataflowHelperFactory iHelperFactory =
+                new IndexDataflowHelperFactory(nc.getStorageManager(), secondaryIndexInfo.getFileSplitProvider());
+        secondaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+        secondaryIndexDataflowHelper.open();
+        secondaryLsmBtree = (TestLsmBtree) secondaryIndexDataflowHelper.getIndexInstance();
+        secondaryIndexDataflowHelper.close();
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsSingleComponent() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsTwoComponents() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            Assert.assertEquals(2, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            id = (LSMComponentId) primaryDiskComponent.getId();
+            min = id.getMaxId();
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsEmpty() throws Exception {
+        try {
+            // ensure primary has no component
+            Assert.assertEquals(0, primaryLsmBtree.getDiskComponents().size());
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(),
+                    latestPrimaryCheckpoint.getLastComponentId());
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsNotEmpty() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+        return ncAppCtx.getIndexCheckpointManagerProvider();
+    }
+
+    private void ensureDone(Actor actor) throws InterruptedException {
+        Request req = new Request(Request.Action.DUMMY);
+        actor.add(req);
+        req.await();
+    }
+
+    private static class Request {
+        enum Action {
+            DUMMY,
+            INSERT_OPEN,
+            LOAD_OPEN,
+            INSERT_PATCH,
+            INDEX_LOAD_PATCH,
+            FLUSH_DATASET,
+            INSERT_CLOSE,
+            LOAD_CLOSE,
+        }
+
+        private final Action action;
+        private volatile boolean done;
+
+        public Request(Action action) {
+            this.action = action;
+            done = false;
+        }
+
+        synchronized void complete() {
+            done = true;
+            notifyAll();
+        }
+
+        synchronized void await() throws InterruptedException {
+            while (!done) {
+                wait();
+            }
+        }
+    }
+
+    public class Actor extends SingleThreadEventProcessor<Request> {
+        private final RecordTupleGenerator primaryInsertTupleGenerator;
+        private final FrameTupleAppender tupleAppender;
+
+        public Actor(String name) throws HyracksDataException {
+            super(name);
+            primaryInsertTupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            tupleAppender = new FrameTupleAppender(new VSizeFrame(taskCtx));
+        }
+
+        @Override
+        protected void handle(Request req) throws Exception {
+            try {
+                switch (req.action) {
+                    case FLUSH_DATASET:
+                        if (tupleAppender.getTupleCount() > 0) {
+                            tupleAppender.write(insertOp, true);
+                        }
+                        dsLifecycleMgr.flushDataset(dataset.getDatasetId(), false);
+                        break;
+                    case INSERT_CLOSE:
+                        insertOp.close();
+                        break;
+                    case INSERT_OPEN:
+                        insertOp.open();
+                        break;
+                    case LOAD_OPEN:
+                        indexLoadOp.open();
+                        break;
+                    case LOAD_CLOSE:
+                        indexLoadOp.close();
+                        break;
+                    case INSERT_PATCH:
+                        for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+                            ITupleReference tuple = primaryInsertTupleGenerator.next();
+                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
+                        }
+                        if (tupleAppender.getTupleCount() > 0) {
+                            tupleAppender.write(insertOp, true);
+                        }
+                        StorageTestUtils.waitForOperations(primaryLsmBtree);
+                        break;
+                    case INDEX_LOAD_PATCH:
+                        TupleGenerator secondaryLoadTupleGenerator =
+                                new TupleGenerator(SECONDARY_INDEX_VALUE_GENERATOR, secondaryIndexInfo.getSerdes(), 0);
+                        FrameTupleAppender secondaryTupleAppender = new FrameTupleAppender(new VSizeFrame(loadTaskCtx));
+                        for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+                            ITupleReference tuple = secondaryLoadTupleGenerator.next();
+                            DataflowUtils.addTupleToFrame(secondaryTupleAppender, tuple, indexLoadOp);
+                        }
+                        if (secondaryTupleAppender.getTupleCount() > 0) {
+                            secondaryTupleAppender.write(indexLoadOp, true);
+                        }
+                        break;
+                    default:
+                        break;
+                }
+            } catch (Throwable th) {
+                th.printStackTrace();
+                throw th;
+            } finally {
+                req.complete();
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
index 9ef531e..017c59f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
@@ -20,16 +20,18 @@
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Predicate;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TransactionOptions;
@@ -50,6 +52,7 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
@@ -119,7 +122,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -146,16 +149,27 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
                     StorageTestUtils.TOTAL_NUM_OF_RECORDS - StorageTestUtils.RECORDS_PER_COMPONENT);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -176,7 +190,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -203,7 +217,13 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -227,9 +247,14 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -247,7 +272,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -278,7 +303,13 @@
             // now that we enetered, we will rollback
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(
                     c -> (c instanceof ILSMMemoryComponent && ((ILSMMemoryComponent) c).isModified()));
@@ -297,10 +328,14 @@
             // wait till firstSearcher enter the components
             secondSearcher.waitUntilEntered();
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             // now that the rollback has completed, we will unblock the search
@@ -321,7 +356,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -376,7 +411,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -407,11 +442,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // now that we enetered, we will rollback
             Rollerback rollerback = new Rollerback(lsmBtree, new DiskComponentLsnPredicate(lsn));
@@ -439,7 +474,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -498,7 +533,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -559,7 +594,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -590,11 +625,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -629,7 +664,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -659,12 +694,12 @@
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             // select the components to merge... the last three
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             int numMergedComponents = 3;
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -704,7 +739,13 @@
                     ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
                     try {
                         dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-                        ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+                        ILSMComponentId next =
+                                dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+                        long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+                        Map<String, Object> flushMap = new HashMap<>();
+                        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+                        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+                        lsmAccessor.getOpContext().setParameters(flushMap);
                         lsmAccessor.deleteComponents(predicate);
                     } catch (HyracksDataException e) {
                         failure = e;
@@ -733,9 +774,8 @@
         @Override
         public boolean test(ILSMComponent c) {
             try {
-                return c instanceof ILSMMemoryComponent
-                        || (c instanceof ILSMDiskComponent && AbstractLSMIOOperationCallback
-                                .getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
+                return c instanceof ILSMMemoryComponent || (c instanceof ILSMDiskComponent
+                        && LSMIOOperationCallback.getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
             } catch (HyracksDataException e) {
                 e.printStackTrace();
                 return false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
index 8bafd32..b618727 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
@@ -22,7 +22,7 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -91,7 +91,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(ILSMMemoryComponent c) throws HyracksDataException {
                 // No Op
             }
         };
@@ -118,7 +118,7 @@
             throws Exception {
         NCAppRuntimeContext ncAppCtx = nc.getAppRuntimeContext();
         IDatasetLifecycleManager dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
-        TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+        RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
         ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                 new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
         boolean failed = false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index dabb32c..79e6368 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.test.dataflow;
 
 import java.io.File;
+import java.lang.reflect.Field;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,7 +29,7 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -60,8 +61,14 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -69,6 +76,7 @@
 import org.junit.Test;
 
 public class LSMFlushRecoveryTest {
+    public static final Logger LOGGER = LogManager.getLogger();
     private static TestNodeController nc;
     private static Dataset dataset;
     private static PrimaryIndexInfo[] primaryIndexInfos;
@@ -84,7 +92,7 @@
     private static IIndexDataflowHelper[] secondaryIndexDataflowHelpers;
     private static ITransactionContext txnCtx;
     private static LSMInsertDeleteOperatorNodePushable[] insertOps;
-    private static TupleGenerator tupleGenerator;
+    private static RecordTupleGenerator tupleGenerator;
 
     private static final int NUM_PARTITIONS = 2;
     private static final int PARTITION_0 = 0;
@@ -153,6 +161,22 @@
     private void initializeNc(boolean cleanUpOnStart) throws Exception {
         nc.init(cleanUpOnStart);
         ncAppCtx = nc.getAppRuntimeContext();
+        // Override the LSMIOScheduler to avoid halting on failure and enable
+        // testing failure scenario in a unit test setting
+        Field ioScheduler = ncAppCtx.getClass().getDeclaredField("lsmIOScheduler");
+        ioScheduler.setAccessible(true);
+        ioScheduler.set(ncAppCtx, new AsynchronousScheduler(ncAppCtx.getServiceContext().getThreadFactory(),
+                new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.error("Scheduler Failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                        LOGGER.warn("IO Operation failed", t);
+                    }
+                }));
         dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
     }
 
@@ -241,21 +265,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore primaryFlushSemaphore = new Semaphore(0);
         primaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        primaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        primaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 primaryFlushSemaphore.release();
             }
         });
@@ -283,21 +307,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore secondaryFlushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 secondaryFlushSemaphore.release();
             }
         });
@@ -335,21 +359,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore flushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 flushSemaphore.release();
             }
         });
@@ -387,7 +411,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
@@ -400,7 +424,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
@@ -454,6 +478,8 @@
             ILSMMemoryComponent primaryMemComponent = primaryIndexes[partitionIndex].getCurrentMemoryComponent();
             ILSMMemoryComponent secondaryMemComponent = secondaryIndexes[partitionIndex].getCurrentMemoryComponent();
             Assert.assertEquals(primaryMemComponent.getId(), secondaryMemComponent.getId());
+            Assert.assertEquals(primaryIndexes[partitionIndex].getCurrentMemoryComponentIndex(),
+                    secondaryIndexes[partitionIndex].getCurrentMemoryComponentIndex());
         }
 
         List<ILSMDiskComponent> primaryDiskComponents = primaryIndexes[partitionIndex].getDiskComponents();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
index 2121327..e2c99b0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
@@ -26,10 +26,9 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
@@ -37,10 +36,7 @@
 import org.apache.asterix.common.transactions.TransactionOptions;
 import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
@@ -113,22 +109,20 @@
             StorageComponentProvider storageManager = new StorageComponentProvider();
             List<List<String>> partitioningKeys = new ArrayList<>();
             partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
-                PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                        storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+                PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE,
+                        META_TYPE, null, storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
                 JobId jobId = nc.newJobId();
                 IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, true);
                 ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp =
+                        nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
                 insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                        RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+                RecordTupleGenerator tupleGenerator =
+                        new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+                                RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
                 VSizeFrame frame = new VSizeFrame(ctx);
                 VSizeFrame marker = new VSizeFrame(ctx);
                 FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
@@ -178,9 +172,9 @@
                 nc.newJobId();
                 TestTupleCounterFrameWriter countOp = create(nc.getSearchOutputDesc(KEY_TYPES, RECORD_TYPE, META_TYPE),
                         Collections.emptyList(), Collections.emptyList(), false);
-                IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, dataset, KEY_TYPES, RECORD_TYPE,
-                        META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES, KEY_INDICATORS_LIST,
-                        storageManager);
+                IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, StorageTestUtils.DATASET, KEY_TYPES,
+                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES,
+                        KEY_INDICATORS_LIST, storageManager);
                 emptyTupleOp.open();
                 emptyTupleOp.close();
                 Assert.assertEquals(NUM_OF_RECORDS, countOp.getCount());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index 62705cc..a7225a1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -29,8 +29,8 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -302,7 +302,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (allocated) {
                         allocated.setValue(true);
                         allocated.notifyAll();
@@ -339,7 +339,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             synchronized (proceedToScheduleFlush) {
@@ -421,7 +421,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (finishedSchduleFlush) {
                         finishedSchduleFlush.set(true);
                         finishedSchduleFlush.notifyAll();
@@ -478,7 +478,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                     synchronized (recycledPrimary) {
                         recycledPrimary.setValue(true);
                         recycledPrimary.notifyAll();
@@ -519,7 +519,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                 }
             };
             secondaryLsmBtrees[0].addIoRecycleCallback(secondaryRecycleCallback);
@@ -619,14 +619,14 @@
 
     public class Actor extends SingleThreadEventProcessor<Request> {
         private final int partition;
-        private final TupleGenerator tupleGenerator;
+        private final RecordTupleGenerator tupleGenerator;
         private final VSizeFrame frame;
         private final FrameTupleAppender tupleAppender;
 
         public Actor(String name, int partition) throws HyracksDataException {
             super(name);
             this.partition = partition;
-            tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+            tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
                     RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             frame = new VSizeFrame(taskCtxs[partition]);
             tupleAppender = new FrameTupleAppender(frame);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index e4373f6..61c1fb2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -26,8 +26,8 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -60,8 +60,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -160,8 +161,8 @@
             // except search
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+                    KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             Searcher firstSearcher = null;
@@ -206,8 +207,8 @@
             // except search
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+                    KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             Searcher firstSearcher = null;
@@ -235,10 +236,11 @@
             // merge all components
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>(lsmBtree.getDiskComponents());
-            BlockingIOOperationCallbackWrapper ioCallback =
-                    new BlockingIOOperationCallbackWrapper(lsmBtree.getIOOperationCallback());
-            mergeAccessor.scheduleMerge(ioCallback, mergedComponents);
-            ioCallback.waitForIO();
+            ILSMIOOperation merge = mergeAccessor.scheduleMerge(mergedComponents);
+            merge.sync();
+            if (merge.getStatus() == LSMIOOperationStatus.FAILURE) {
+                throw HyracksDataException.create(merge.getFailure());
+            }
             // unblock the search
             unblockSearch(lsmBtree);
             // ensure the search got the correct number
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
index 390286a..589e8b2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
@@ -33,9 +33,9 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.DatasetInfo;
@@ -100,7 +100,7 @@
     private StorageTestUtils() {
     }
 
-    static void allowAllOps(TestLsmBtree lsmBtree) {
+    public static void allowAllOps(TestLsmBtree lsmBtree) {
         lsmBtree.clearModifyCallbacks();
         lsmBtree.clearFlushCallbacks();
         lsmBtree.clearSearchCallbacks();
@@ -118,6 +118,12 @@
                 KEY_INDICATORS_LIST, partition);
     }
 
+    public static PrimaryIndexInfo createPrimaryIndex(TestNodeController nc, Dataset dataset, int partition)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, STORAGE_MANAGER, KEY_INDEXES,
+                KEY_INDICATORS_LIST, partition);
+    }
+
     public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx)
             throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
         return getInsertPipeline(nc, ctx, null);
@@ -131,13 +137,27 @@
     }
 
     public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+            Dataset dataset, Index secondaryIndex, IndexOperation op)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex, op).getLeft();
+    }
+
+    public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
             Index secondaryIndex) throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
         return nc.getInsertPipeline(ctx, DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
                 KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
     }
 
-    public static TupleGenerator getTupleGenerator() {
-        return new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
+    public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+            Dataset dataset, Index secondaryIndex)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
+    }
+
+    public static RecordTupleGenerator getTupleGenerator() {
+        return new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
                 UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
     }
 
@@ -146,6 +166,11 @@
         searchAndAssertCount(nc, partition, DATASET, STORAGE_MANAGER, numOfRecords);
     }
 
+    public static void searchAndAssertCount(TestNodeController nc, Dataset dataset, int partition, int numOfRecords)
+            throws HyracksDataException, AlgebricksException {
+        searchAndAssertCount(nc, partition, dataset, STORAGE_MANAGER, numOfRecords);
+    }
+
     public static void searchAndAssertCount(TestNodeController nc, int partition, Dataset dataset,
             StorageComponentProvider storageManager, int numOfRecords)
             throws HyracksDataException, AlgebricksException {
@@ -182,6 +207,11 @@
         flushPartition(dslLifecycleMgr, lsmBtree, DATASET, async);
     }
 
+    public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+            boolean async) throws Exception {
+        flushPartition(dslLifecycleMgr, lsmBtree, dataset, async);
+    }
+
     public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
             boolean async) throws Exception {
         waitForOperations(lsmBtree);
@@ -211,6 +241,11 @@
         flush(dsLifecycleMgr, lsmBtree, DATASET, async);
     }
 
+    public static void flush(IDatasetLifecycleManager dsLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+            boolean async) throws Exception {
+        flush(dsLifecycleMgr, lsmBtree, dataset, async);
+    }
+
     public static void flush(IDatasetLifecycleManager dsLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
             boolean async) throws Exception {
         waitForOperations(lsmBtree);
@@ -240,6 +275,11 @@
             this(nc, partition, DATASET, STORAGE_MANAGER, lsmBtree, numOfRecords);
         }
 
+        public Searcher(TestNodeController nc, Dataset dataset, int partition, TestLsmBtree lsmBtree,
+                int numOfRecords) {
+            this(nc, partition, dataset, STORAGE_MANAGER, lsmBtree, numOfRecords);
+        }
+
         public Searcher(TestNodeController nc, int partition, Dataset dataset, StorageComponentProvider storageManager,
                 TestLsmBtree lsmBtree, int numOfRecords) {
             lsmBtree.addSearchCallback(new ITestOpCallback<Semaphore>() {
@@ -253,7 +293,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             Callable<Boolean> callable = new Callable<Boolean>() {
@@ -292,7 +332,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
@@ -319,7 +359,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index 0c4983a..bcf68b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -21,6 +21,7 @@
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -28,9 +29,14 @@
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
+import org.apache.asterix.transaction.management.runtime.CommitRuntime;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.common.IResourceFactory;
@@ -48,6 +54,19 @@
     }
 
     @Override
+    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider,
+            int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
+        return new IPushRuntimeFactory() {
+            @Override
+            public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IPushRuntime[] { new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()),
+                        getDatasetId(), primaryKeyFieldPermutation, true,
+                        ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
+            }
+        };
+    }
+
+    @Override
     public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Index index, ARecordType recordType,
             ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties)
             throws AlgebricksException {
@@ -62,6 +81,6 @@
 
     @Override
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        return new TestLsmBtreeIoOpCallbackFactory(getComponentIdGeneratorFactory());
+        return new TestLsmIoOpCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
deleted file mode 100644
index 5852ad9..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.test.dataflow;
-
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
-
-public class TestLsmBtreeIoOpCallbackFactory extends LSMBTreeIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private static volatile int completedFlushes = 0;
-    private static volatile int completedMerges = 0;
-    private static volatile int rollbackFlushes = 0;
-    private static volatile int rollbackMerges = 0;
-    private static volatile int failedFlushes = 0;
-    private static volatile int failedMerges = 0;
-
-    public TestLsmBtreeIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        completedFlushes = 0;
-        completedMerges = 0;
-        rollbackFlushes = 0;
-        rollbackMerges = 0;
-        // Whenever this is called, it resets the counter
-        // However, the counters for the failed operations are never reset since we expect them
-        // To be always 0
-        return new TestLsmBtreeIoOpCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-
-    public int getTotalFlushes() {
-        return completedFlushes + rollbackFlushes;
-    }
-
-    public int getTotalMerges() {
-        return completedMerges + rollbackMerges;
-    }
-
-    public int getTotalIoOps() {
-        return getTotalFlushes() + getTotalMerges();
-    }
-
-    public int getRollbackFlushes() {
-        return rollbackFlushes;
-    }
-
-    public int getRollbackMerges() {
-        return rollbackMerges;
-    }
-
-    public int getCompletedFlushes() {
-        return completedFlushes;
-    }
-
-    public int getCompletedMerges() {
-        return completedMerges;
-    }
-
-    public static int getFailedFlushes() {
-        return failedFlushes;
-    }
-
-    public static int getFailedMerges() {
-        return failedMerges;
-    }
-
-    public class TestLsmBtreeIoOpCallback extends LSMBTreeIOOperationCallback {
-        private final TestLsmBtree lsmBtree;
-
-        public TestLsmBtreeIoOpCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-                IIndexCheckpointManagerProvider checkpointManagerProvider) {
-            super(index, idGenerator, checkpointManagerProvider);
-            lsmBtree = (TestLsmBtree) index;
-        }
-
-        @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.beforeIoOperationCalled();
-            super.beforeOperation(opCtx);
-            lsmBtree.beforeIoOperationReturned();
-        }
-
-        @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.afterIoOperationCalled();
-            super.afterOperation(opCtx);
-            lsmBtree.afterIoOperationReturned();
-        }
-
-        @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.afterIoFinalizeCalled();
-            super.afterFinalize(opCtx);
-            synchronized (TestLsmBtreeIoOpCallbackFactory.this) {
-                if (opCtx.getNewComponent() != null) {
-                    if (opCtx.getNewComponent() == EmptyComponent.INSTANCE) {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-                            rollbackFlushes++;
-                        } else {
-                            rollbackMerges++;
-                        }
-                    } else {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-                            completedFlushes++;
-                        } else {
-                            completedMerges++;
-                        }
-                    }
-                } else {
-                    recordFailure(opCtx.getIoOperationType());
-                }
-                TestLsmBtreeIoOpCallbackFactory.this.notifyAll();
-            }
-            lsmBtree.afterIoFinalizeReturned();
-        }
-
-        @Override
-        public void recycled(ILSMMemoryComponent component, boolean advance) throws HyracksDataException {
-            lsmBtree.recycledCalled(component);
-            super.recycled(component, advance);
-            lsmBtree.recycledReturned(component);
-        }
-
-        @Override
-        public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-            lsmBtree.allocatedCalled(component);
-            super.allocated(component);
-            lsmBtree.allocatedReturned(component);
-        }
-
-        private void recordFailure(LSMIOOperationType opType) {
-            if (opType == LSMIOOperationType.FLUSH) {
-                failedFlushes++;
-            } else {
-                failedMerges++;
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
new file mode 100644
index 0000000..c762c8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.dataflow;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
+
+public class TestLsmIoOpCallbackFactory extends LSMIndexIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private static volatile int completedFlushes = 0;
+    private static volatile int completedMerges = 0;
+    private static volatile int rollbackFlushes = 0;
+    private static volatile int rollbackMerges = 0;
+    private static volatile int failedFlushes = 0;
+    private static volatile int failedMerges = 0;
+
+    public TestLsmIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        super(idGeneratorFactory, datasetInfoProvider);
+    }
+
+    @Override
+    public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        completedFlushes = 0;
+        completedMerges = 0;
+        rollbackFlushes = 0;
+        rollbackMerges = 0;
+        // Whenever this is called, it resets the counter
+        // However, the counters for the failed operations are never reset since we expect them
+        // To be always 0
+        return new TestLsmIoOpCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+                getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+    }
+
+    public int getTotalFlushes() {
+        return completedFlushes + rollbackFlushes;
+    }
+
+    public int getTotalMerges() {
+        return completedMerges + rollbackMerges;
+    }
+
+    public int getTotalIoOps() {
+        return getTotalFlushes() + getTotalMerges();
+    }
+
+    public int getRollbackFlushes() {
+        return rollbackFlushes;
+    }
+
+    public int getRollbackMerges() {
+        return rollbackMerges;
+    }
+
+    public int getCompletedFlushes() {
+        return completedFlushes;
+    }
+
+    public int getCompletedMerges() {
+        return completedMerges;
+    }
+
+    public static int getFailedFlushes() {
+        return failedFlushes;
+    }
+
+    public static int getFailedMerges() {
+        return failedMerges;
+    }
+
+    public class TestLsmIoOpCallback extends LSMIOOperationCallback {
+        private final TestLsmBtree lsmBtree;
+
+        public TestLsmIoOpCallback(DatasetInfo dsInfo, ILSMIndex index, ILSMComponentId id,
+                IIndexCheckpointManagerProvider checkpointManagerProvider) {
+            super(dsInfo, index, id, checkpointManagerProvider);
+            lsmBtree = (TestLsmBtree) index;
+        }
+
+        @Override
+        public void scheduled(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.ioScheduledCalled();
+            super.scheduled(op);
+            lsmBtree.ioScheduledReturned();
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.beforeIoOperationCalled();
+            super.beforeOperation(op);
+            lsmBtree.beforeIoOperationReturned();
+        }
+
+        @Override
+        public void afterOperation(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.afterIoOperationCalled();
+            super.afterOperation(op);
+            lsmBtree.afterIoOperationReturned();
+        }
+
+        @Override
+        public void afterFinalize(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.afterIoFinalizeCalled();
+            super.afterFinalize(op);
+            synchronized (TestLsmIoOpCallbackFactory.this) {
+                if (op.getNewComponent() != null) {
+                    if (op.getNewComponent() == EmptyComponent.INSTANCE) {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+                            rollbackFlushes++;
+                        } else {
+                            rollbackMerges++;
+                        }
+                    } else {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+                            completedFlushes++;
+                        } else {
+                            completedMerges++;
+                        }
+                    }
+                } else {
+                    recordFailure(op.getIOOpertionType());
+                }
+                TestLsmIoOpCallbackFactory.this.notifyAll();
+            }
+            lsmBtree.afterIoFinalizeReturned();
+        }
+
+        @Override
+        public void completed(ILSMIOOperation operation) {
+            try {
+                lsmBtree.ioCompletedCalled();
+                super.completed(operation);
+                lsmBtree.ioCompletedReturned();
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        @Override
+        public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+            lsmBtree.recycledCalled(component);
+            super.recycled(component);
+            lsmBtree.recycledReturned(component);
+        }
+
+        @Override
+        public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+            lsmBtree.allocatedCalled(component);
+            super.allocated(component);
+            lsmBtree.allocatedReturned(component);
+        }
+
+        private void recordFailure(LSMIOOperationType opType) {
+            if (opType == LSMIOOperationType.FLUSH) {
+                failedFlushes++;
+            } else {
+                failedMerges++;
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
index 9a528d3..250c25f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
@@ -60,7 +60,7 @@
         super.triggerScheduleFlush(logRecord);
         synchronized (callbacks) {
             for (ITestOpCallback<Void> callback : callbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
index 06b213d..bee2f8d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
@@ -23,7 +23,7 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -68,7 +68,7 @@
     private static IHyracksTaskContext abortCtx;
     private static ITransactionContext abortTxnCtx;
     private static LSMInsertDeleteOperatorNodePushable abortOp;
-    private static TupleGenerator tupleGenerator;
+    private static RecordTupleGenerator tupleGenerator;
 
     @Rule
     public TestRule watcher = new TestMethodTracer();
@@ -199,7 +199,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
                 // manually set the current memory component as modified
                 index.getCurrentMemoryComponent().setModified();
                 throw new HyracksDataException("Fail the job");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
index 0affaf2..726a1bf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
@@ -89,7 +89,7 @@
             index = metadataProvider.getIndex(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName,
                     primaryIndexName);
             Assert.assertNotNull(index);
-            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider);
+            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
             jobSpecification.getOperatorMap().values().forEach(iOperatorDescriptor -> {
                 Assert.assertFalse(iOperatorDescriptor instanceof AbstractSorterOperatorDescriptor);
             });
@@ -99,7 +99,7 @@
             index = metadataProvider.getIndex(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName,
                     secondaryIndexName);
             Assert.assertNotNull(index);
-            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider);
+            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
             final long numOfSortOperators = jobSpecification.getOperatorMap().values().stream()
                     .filter(op -> op instanceof AbstractSorterOperatorDescriptor).count();
             Assert.assertTrue(numOfSortOperators != 0);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
index 418282e..3634bf1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.test.logging;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
 import java.io.File;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -26,10 +30,9 @@
 import java.util.List;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.RecoveryManager;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.Checkpoint;
@@ -43,14 +46,12 @@
 import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.test.common.TestHelper;
+import org.apache.asterix.test.dataflow.StorageTestUtils;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
 import org.apache.asterix.transaction.management.service.recovery.AbstractCheckpointManager;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
@@ -60,17 +61,12 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.stubbing.Answer;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.spy;
-
 public class CheckpointingTest {
 
     private static final String TEST_CONFIG_FILE_NAME = "cc-small-txn-log-partition.conf";
@@ -116,23 +112,21 @@
             nc.init();
             List<List<String>> partitioningKeys = new ArrayList<>();
             partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
-                nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
-                        KEY_INDICATOR_LIST, 0);
+                nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+                        KEY_INDEXES, KEY_INDICATOR_LIST, 0);
                 JobId jobId = nc.newJobId();
                 IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
                 ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
                 // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp =
+                        nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
                 insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
-                        RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+                RecordTupleGenerator tupleGenerator =
+                        new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
+                                RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
                 VSizeFrame frame = new VSizeFrame(ctx);
                 FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
 
@@ -197,8 +191,9 @@
                 nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx2),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
                 // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp2 = nc.getInsertPipeline(ctx2, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp2 =
+                        nc.getInsertPipeline(ctx2, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
                 insertOp2.open();
                 VSizeFrame frame2 = new VSizeFrame(ctx2);
                 FrameTupleAppender tupleAppender2 = new FrameTupleAppender(frame2);
@@ -220,6 +215,7 @@
                     }
                 }
                 Thread.UncaughtExceptionHandler h = new Thread.UncaughtExceptionHandler() {
+                    @Override
                     public void uncaughtException(Thread th, Throwable ex) {
                         threadException = true;
                         exception = ex;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
index 0f6adf6..d5985b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -51,7 +51,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.test.support.TestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -261,8 +260,9 @@
         final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxn);
         final String nodeGroupName = "ng";
+        final String committedNodeGroup = "committed_ng";
+        final List<String> ngNodes = Collections.singletonList("asterix_nc1");
         try {
-            final List<String> ngNodes = Collections.singletonList("asterix_nc1");
             MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup(nodeGroupName, ngNodes));
             MetadataManager.INSTANCE.commitTransaction(mdTxn);
         } finally {
@@ -281,6 +281,9 @@
         int diskComponentsBeforeFlush = index.getDiskComponents().size();
         // lock opTracker to prevent log flusher from triggering flush
         synchronized (opTracker) {
+            final MetadataTransactionContext committedMdTxn = MetadataManager.INSTANCE.beginTransaction();
+            MetadataManager.INSTANCE.addNodegroup(committedMdTxn, new NodeGroup(committedNodeGroup, ngNodes));
+            MetadataManager.INSTANCE.commitTransaction(committedMdTxn);
             opTracker.setFlushOnExit(true);
             opTracker.flushIfNeeded();
             Assert.assertTrue(opTracker.isFlushLogCreated());
@@ -288,7 +291,6 @@
             // make sure force operation will processed
             MetadataManager.INSTANCE.dropNodegroup(mdTxn2, nodeGroupName, false);
             Assert.assertEquals(1, opTracker.getNumActiveOperations());
-            Assert.assertFalse(index.hasFlushRequestForCurrentMutableComponent());
             // release opTracker lock now to allow log flusher to schedule the flush
             InvokeUtil.runWithTimeout(() -> {
                 synchronized (opTracker) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
deleted file mode 100644
index 09758f6..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with full parallelism on node controllers.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionFullParallelismIT {
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc2.conf";
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        LangExecutionUtil.tearDown();
-    }
-
-    @Parameters(name = "AqlExecutionFullParallelismIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
-        if (!tests.isEmpty()) {
-            tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
-        } else {
-            tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
-            tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
-        }
-        return tests;
-    }
-
-    protected TestCaseContext tcCtx;
-
-    public AqlExecutionFullParallelismIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Test
-    public void test() throws Exception {
-        LangExecutionUtil.test(tcCtx);
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
deleted file mode 100644
index 62ed790..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with less parallelism on node controllers than using all the cores.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionLessParallelismIT {
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc3.conf";
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        LangExecutionUtil.tearDown();
-    }
-
-    @Parameters(name = "AqlExecutionLessParallelismIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
-        if (!tests.isEmpty()) {
-            tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
-        } else {
-            tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
-            tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
-        }
-        return tests;
-    }
-
-    protected TestCaseContext tcCtx;
-
-    public AqlExecutionLessParallelismIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Test
-    public void test() throws Exception {
-        LangExecutionUtil.test(tcCtx);
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index c7ae2df..62c882d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -107,6 +106,10 @@
         tearDown(cleanup, integrationUtil, true);
     }
 
+    public static void tearDown(boolean cleanup, boolean stopHdfs) throws Exception {
+        tearDown(cleanup, integrationUtil, stopHdfs);
+    }
+
     public static void tearDown(boolean cleanup, AsterixHyracksIntegrationUtil integrationUtil, boolean stopHdfs)
             throws Exception {
         // validateBufferCacheState(); <-- Commented out until bug is fixed -->
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
deleted file mode 100644
index 17509a4..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.test.storage;
-
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
-import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.TransactionOptions;
-import org.apache.asterix.external.util.DataflowUtils;
-import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.util.DiskUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DiskIsFullTest {
-
-    private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
-    private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
-            new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
-    private static final GenerationFunction[] RECORD_GEN_FUNCTION =
-            { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
-    private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
-    private static final ARecordType META_TYPE = null;
-    private static final GenerationFunction[] META_GEN_FUNCTION = null;
-    private static final boolean[] UNIQUE_META_FIELDS = null;
-    private static final int[] KEY_INDEXES = { 0 };
-    private static final int[] KEY_INDICATOR = { Index.RECORD_INDICATOR };
-    private static final List<Integer> KEY_INDICATOR_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
-    private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
-    private static final String DATASET_NAME = "TestDS";
-    private static final String DATA_TYPE_NAME = "DUMMY";
-    private static final String NODE_GROUP_NAME = "DEFAULT";
-    private static final String TEST_DISK_NAME = "asterixdb_ram_disk";
-    private boolean shouldRun = true;
-
-    @Before
-    public void setUp() throws Exception {
-        if (!SystemUtils.IS_OS_MAC) {
-            System.out.println("Skipping test " + DiskIsFullTest.class.getName() + " due to unsupported OS");
-            shouldRun = false;
-            return;
-        }
-        System.out.println("SetUp: ");
-        TestHelper.deleteExistingInstanceFiles();
-        // create RAM disk
-        final Path ramDiskRoot = DiskUtil.mountRamDisk(TEST_DISK_NAME, 4, MEGABYTE);
-        // Use RAM disk for storage
-        AsterixHyracksIntegrationUtil.setStoragePath(ramDiskRoot.toAbsolutePath().toString());
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        if (!shouldRun) {
-            return;
-        }
-        System.out.println("TearDown");
-        TestHelper.deleteExistingInstanceFiles();
-        DiskUtil.unmountRamDisk(TEST_DISK_NAME);
-        AsterixHyracksIntegrationUtil.restoreDefaultStoragePath();
-    }
-
-    @Test
-    public void testDiskIsFull() {
-        if (!shouldRun) {
-            return;
-        }
-        HyracksDataException expectedException =
-                HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-        try {
-            TestNodeController nc = new TestNodeController(null, false);
-            nc.init();
-            StorageComponentProvider storageManager = new StorageComponentProvider();
-            List<List<String>> partitioningKeys = new ArrayList<>();
-            partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
-            try {
-                nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
-                        KEY_INDICATOR_LIST, 0);
-                JobId jobId = nc.newJobId();
-                IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
-                ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
-                        new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
-                // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
-                try {
-                    insertOp.open();
-                    TupleGenerator tupleGenerator =
-                            new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR, RECORD_GEN_FUNCTION,
-                                    UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
-                    VSizeFrame frame = new VSizeFrame(ctx);
-                    FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
-                    // Insert records until disk becomes full
-                    int tupleCount = 100000;
-                    while (tupleCount > 0) {
-                        ITupleReference tuple = tupleGenerator.next();
-                        try {
-                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
-                        } catch (Throwable t) {
-                            final Throwable rootCause = ExceptionUtils.getRootCause(t);
-                            rootCause.printStackTrace();
-                            if (rootCause instanceof HyracksDataException) {
-                                HyracksDataException cause = (HyracksDataException) rootCause;
-                                Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
-                                Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
-                                return;
-                            } else {
-                                break;
-                            }
-                        }
-                        tupleCount--;
-                    }
-                    Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-                } finally {
-                    try {
-                        insertOp.close();
-                    } finally {
-                        nc.getTransactionManager().abortTransaction(txnCtx.getTxnId());
-                    }
-                }
-            } finally {
-                nc.deInit();
-            }
-        } catch (Throwable e) {
-            e.printStackTrace();
-            Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
index 1c8ac62..4453a1d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.config.TransactionProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
@@ -64,6 +65,8 @@
     @Before
     public void setUp() throws Exception {
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+        // use a small page size for test purpose
+        integrationUtil.addOption(TransactionProperties.Option.TXN_LOG_BUFFER_PAGESIZE, 128 * 1024);
         integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
     }
 
diff --git a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
index 6cd3b87..811a40d 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
@@ -52,5 +52,6 @@
 messaging.frame.size=4096
 messaging.frame.count=512
 txn.log.partitionsize=2MB
+txn.log.buffer.pagesize=128KB
 txn.log.checkpoint.pollfrequency=2147483647
 storage.max.active.writable.datasets=50
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 822d2c3..5fc2c1e 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -50,6 +50,7 @@
 { "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "geometry", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index c0ab7a4..e47c518 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -339,73 +339,73 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_3">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_4">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_5">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_6">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_7">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_8">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_251_dataset_hint_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
+        <expected-error>Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_251_dataset_hint_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
+        <expected-error>Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_272_create_index_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+        <expected-error>Cannot find dataset with name Foo in dataverse test</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_272_create_index_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+        <expected-error>Cannot find dataset with name Foo in dataverse test</expected-error>
       </compilation-unit>
     </test-case>
     <!-- This case should be fixed to return a proper message rather than NPE -->
@@ -444,37 +444,37 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[age]" which is of type integer cannot be indexed using the RTree index.</expected-error>
+        <expected-error>The field "[age]" which is of type integer cannot be indexed using the RTree index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_3">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_4">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_5">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_6">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -482,7 +482,7 @@
     <test-case FilePath="transaction">
       <compilation-unit name="failure_previous_success">
         <output-dir compare="Text">failure_previous_success</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A datatype with this name StreetType already exists</expected-error>
+        <expected-error>A datatype with this name StreetType already exists</expected-error>
       </compilation-unit>
       <compilation-unit name="verify_failure_previous_success">
         <output-dir compare="Text">verify_failure_previous_success</output-dir>
@@ -491,11 +491,11 @@
     <test-case FilePath="transaction">
       <compilation-unit name="failure_subsequent_no_execution">
         <output-dir compare="Text">failure_subsequent_no_execution</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A nodegroup with this name group1 already exists</expected-error>
+        <expected-error>A nodegroup with this name group1 already exists</expected-error>
       </compilation-unit>
       <compilation-unit name="verify_failure_subsequent_no_execution">
         <output-dir compare="Text">verify_failure_subsequent_no_execution</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
-</test-suite>
+</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
index 88f8cd8..48e3a73 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
@@ -31,6 +31,6 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-COLL_COUNT(FacebookUsers);
+STRICT_COUNT(FacebookUsers);
 
-drop dataverse TinySocial;
\ No newline at end of file
+drop dataverse TinySocial;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
index 01a85d8..ae31568 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
@@ -31,5 +31,5 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-SELECT count(`friend-ids`) * avg(coll_count(`friend-ids`)) AS total
+SELECT count(`friend-ids`) * avg(strict_count(`friend-ids`)) AS total
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
index 50b799e..57ead72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
@@ -33,5 +33,5 @@
 
 // The plan uses a group-by operator with the groupAll flag be true.
 WITH t AS 1
-SELECT count(`friend-ids`) * avg(coll_count(`friend-ids`)) AS total, t
+SELECT count(`friend-ids`) * avg(strict_count(`friend-ids`)) AS total, t
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
index 8f7a024..d49c51a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
@@ -31,5 +31,5 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`))
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`))
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
deleted file mode 100644
index e757455..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"count1":$t1.countA,
-"t2info": for $t2 in dataset('TweetMessages')
-                        where $t1.countA /* +indexnl */= $t2.countB and
-                        $t1.tweetid != $t2.tweetid
-                        order by $t2.tweetid
-                        return {"tweetid2": $t2.tweetid,
-                                       "count2":$t2.countB}
-};
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
new file mode 100644
index 0000000..96e43a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
deleted file mode 100644
index 1775c6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
- *                  Given the 'indexnl' hint we expect the join to be transformed
- *                  into an indexed nested-loop join using Customers' primary index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  lastorder: {
-    oid: int32,
-    total: float
-  }
-}
-
-create type OrderType as closed {
-  oid: int32,
-  cid: int32,
-  orderstatus: string,
-  orderpriority: string,
-  clerk: string,
-  total: float
-}
-
-create dataset Customers(CustomerType) primary key cid;
-create dataset Orders(OrderType) primary key oid;
-
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
-
-for $o in dataset('Orders')
-for $c in dataset('Customers')
-where $o.cid /*+ indexnl */ = $c.cid 
-return {"customer":$c, "order": $o} 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
new file mode 100644
index 0000000..e465dc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create type test.OrderType as
+ closed {
+  oid : integer,
+  cid : integer,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Orders(OrderType) primary key oid;
+
+write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
+select element {'customer':c,'order':o}
+from  Orders as o,
+      Customers as c
+where (o.cid /*+ indexnl */  = c.cid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
deleted file mode 100644
index bb4ad1b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
- *                  We first expect FacebookUsers' primary index to be used 
- *                  to satisfy the range condition on it's primary key. 
- *                  FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type EmploymentType as closed { 
-  organization-name: string,
-  start-date: date,
-  end-date: date?
-} 
-
-create type FacebookUserType as closed { 
-  id: int32,
-  id-copy: int32,
-  alias: string,
-  name: string,
-  user-since: datetime,
-  user-since-copy: datetime,
-  friend-ids: {{ int32 }},
-  employment: [EmploymentType]
-} 
-
-create type FacebookMessageType as closed { 
-  message-id: int32,
-  message-id-copy: int32,
-  author-id: int32,
-  author-id-copy: int32,
-  in-response-to: int32?,
-  sender-location: point?,
-  message: string
-} 
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookMessages(FacebookMessageType)
-primary key message-id;
-
-create index fbmIdxAutId if not exists on FacebookMessages(author-id-copy);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
-
-for $user in dataset('FacebookUsers')
-for $message in dataset('FacebookMessages')
-where $user.id /*+ indexnl */ = $message.author-id-copy
-and $user.id >= 11000 and $user.id <= 12000
-return {
-  "fbu-ID": $user.id,
-  "fbm-auth-ID": $message.author-id,
-  "uname": $user.name,
-  "message": $message.message
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
new file mode 100644
index 0000000..4580dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
+ *                  We first expect FacebookUsers' primary index to be used
+ *                  to satisfy the range condition on it's primary key.
+ *                  FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.EmploymentType as
+ closed {
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type test.FacebookUserType as
+ closed {
+  id : integer,
+  `id-copy` : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `user-since-copy` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create type test.FacebookMessageType as
+ closed {
+  `message-id` : integer,
+  `message-id-copy` : integer,
+  `author-id` : integer,
+  `author-id-copy` : integer,
+  `in-response-to` : integer?,
+  `sender-location` : point?,
+  message : string
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
+create  dataset FacebookMessages(FacebookMessageType) primary key `message-id`;
+
+create  index fbmIdxAutId  if not exists  on FacebookMessages (`author-id-copy`) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
+select element {'fbu-ID':user.id,'fbm-auth-ID':message.`author-id`,'uname':user.name,'message':message.message}
+from  FacebookUsers as user,
+      FacebookMessages as message
+where ((user.id /*+ indexnl */  = message.`author-id-copy`) and (user.id >= 11000) and (user.id <= 12000))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
deleted file mode 100644
index 5bdacc3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  CSX has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index title_index on CSX(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
new file mode 100644
index 0000000..1c315f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their title.
+ *                  CSX has a secondary btree index on title, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index title_index  on CSX (title) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (a.title /*+ indexnl */  = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
deleted file mode 100644
index 33bb320..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Equi self-joins a dataset, DBLP, based on its title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index title_index on DBLP(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
new file mode 100644
index 0000000..7ab67fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Equi self-joins a dataset, DBLP, based on its title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index title_index  on DBLP (title) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (a.title /*+ indexnl */  = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
deleted file mode 100644
index b72dbcb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Please note this is a Positive test and the BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Roger"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
new file mode 100644
index 0000000..845da2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname > 'Roger')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
deleted file mode 100644
index 0030ddf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// This is a Positive test - prefix search, BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Susan"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
new file mode 100644
index 0000000..db55b28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname >= 'Susan')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
deleted file mode 100644
index 48b67d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification (usage) test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Isa"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
new file mode 100644
index 0000000..ecac451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification (usage) test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname < 'Isa')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
deleted file mode 100644
index 5e39552..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used in query plan
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Vanpatten"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
new file mode 100644
index 0000000..6354f25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname <= 'Vanpatten')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
deleted file mode 100644
index 23518e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used in the optimized query plan 
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Young Seok" and $emp.lname = "Kim"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
new file mode 100644
index 0000000..d8a95cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used in the optimized query plan
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Young Seok') and (emp.lname = 'Kim'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
deleted file mode 100644
index 81fb7b4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Alex" and $emp.lname < "Zach"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
new file mode 100644
index 0000000..75ccb90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Alex') and (emp.lname < 'Zach'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
deleted file mode 100644
index fbb504d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Allan" and $emp.lname < "Zubi"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
new file mode 100644
index 0000000..2cf7dbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname < 'Zubi'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
deleted file mode 100644
index fbc0dbb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Allan" and $emp.lname = "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
new file mode 100644
index 0000000..98f037e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
deleted file mode 100644
index 356503f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Julio" and $emp.lname < "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
new file mode 100644
index 0000000..a88838f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname < 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
deleted file mode 100644
index 9dcc4eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Michael" and $emp.lname <= "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
new file mode 100644
index 0000000..524b0a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Michael') and (emp.lname <= 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
deleted file mode 100644
index 5ffc387..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
new file mode 100644
index 0000000..f6a5120
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
deleted file mode 100644
index 4794082..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
new file mode 100644
index 0000000..cd4fae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
deleted file mode 100644
index 47cc065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Craig" and $emp.lname > "Kevin"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
new file mode 100644
index 0000000..9efcfc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname <= 'Craig') and (emp.lname > 'Kevin'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
deleted file mode 100644
index 9be2c17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
new file mode 100644
index 0000000..3b3b5e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
deleted file mode 100644
index 807d861..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
new file mode 100644
index 0000000..d6d977c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
deleted file mode 100644
index a7492fa..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Craig" and $emp.lname <= "Kevin" and $emp.fname <= "Mary" and $emp.lname >= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
new file mode 100644
index 0000000..985cde7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname <= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname >= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
deleted file mode 100644
index 7066b35..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
- 
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Max" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
new file mode 100644
index 0000000..40ae7d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname > 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
deleted file mode 100644
index 19168f0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Sofia" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
new file mode 100644
index 0000000..8b9a588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname >= 'Sofia')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
deleted file mode 100644
index 09ad8c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Chen" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
new file mode 100644
index 0000000..14d1ce9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname < 'Chen')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
deleted file mode 100644
index 9c4d2ca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Julio" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
new file mode 100644
index 0000000..4070114
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname <= 'Julio')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
deleted file mode 100644
index b49db5c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the primary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Neil" and $emp.fname < "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
new file mode 100644
index 0000000..e3f42ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the primary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Neil') and (emp.fname < 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
deleted file mode 100644
index 479740f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Max" and $emp.fname <= "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
new file mode 100644
index 0000000..8656ab8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
deleted file mode 100644
index 02ad02e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
- 
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Max" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
new file mode 100644
index 0000000..38fad40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname = 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
deleted file mode 100644
index 59801d3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType)  primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
new file mode 100644
index 0000000..e8d5627
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
deleted file mode 100644
index 90ad451..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Julio" and $emp.lname > "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
new file mode 100644
index 0000000..ce68496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 11th Nov 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname > 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
deleted file mode 100644
index 6b6bce0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Julio" and $emp.lname = "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
new file mode 100644
index 0000000..0ef6c12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 11th Nov 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname < 'Julio') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
deleted file mode 100644
index 8b8acaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse colocated if exists;
-
-create dataverse colocated;
-
-use dataverse colocated;
-
-create type UserType as open {
-  uid: int32, 
-  name: string,
-  lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
-  vid: int32, 
-  name: string,
-  lottery_numbers: {{int32}}
-}
-
-create dataset Users(UserType) 
-  primary key uid;
-
-create dataset Visitors(VisitorType) 
-  primary key vid;
-
-
-write output to asterix_nc1:"/tmp/fuzzy1.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.uid = $visitor.vid
-return { "user_name" : $user.name, "visitor_name" : $visitor.name }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
new file mode 100644
index 0000000..ee49511
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse colocated if exists;
+create  dataverse colocated;
+
+use colocated;
+
+
+create type colocated.UserType as
+{
+  uid : integer,
+  name : string,
+  lottery_numbers : {{integer}}
+};
+
+create type colocated.VisitorType as
+{
+  vid : integer,
+  name : string,
+  lottery_numbers : {{integer}}
+};
+
+create  dataset Users(UserType) primary key uid;
+
+create  dataset Visitors(VisitorType) primary key vid;
+
+write output to asterix_nc1:"/tmp/fuzzy1.adm";
+select element {'user_name':user.name,'visitor_name':visitor.name}
+from  Users as user,
+      Visitors as visitor
+where (user.uid = visitor.vid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
deleted file mode 100644
index 2ad271e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse group_no_agg if exists;
-
-create dataverse group_no_agg;
-
-use dataverse group_no_agg;
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType) 
-  primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/.adm";
-
-for $c in dataset('Customers')
-group by $name := $c.c_name  with $c
-return $name
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
new file mode 100644
index 0000000..3c82655
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse group_no_agg if exists;
+create  dataverse group_no_agg;
+
+use group_no_agg;
+
+
+create type group_no_agg.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key c_custkey on group1;
+
+write output to asterix_nc1:"/tmp/.adm";
+select element name
+from  Customers as c
+group by c.c_name as name
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
deleted file mode 100644
index dbaf7be..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse custorder if exists;
-
-create dataverse custorder;
-
-use dataverse custorder;
-
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  lastorder: {
-    oid: int32,
-    total: float
-  }
-}
-
-
-
-create type OrderType as closed {
-  oid: int32,
-  cid: int32,
-  orderstatus: string,
-  orderpriority: string,
-  clerk: string,
-  total: float
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-create dataset Orders(OrderType)
-  primary key oid on group1;
-
-write output to asterix_nc1:"/tmp/custorder.adm";
-
-for $c in dataset('Customers')
-for $o in dataset('Orders')
-where $c.cid = $o.cid
-group by $cid := $c.cid decor $cust := $c with $o
-return {"cid":$cid, "cust":$cust, "cnt-orders": count($o), "orders":$o} 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
new file mode 100644
index 0000000..508d6ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse custorder if exists;
+create  dataverse custorder;
+
+use custorder;
+
+
+create type custorder.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type custorder.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create type custorder.OrderType as
+ closed {
+  oid : integer,
+  cid : integer,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key cid on group1;
+
+create  dataset Orders(OrderType) primary key oid on group1;
+
+write output to asterix_nc1:"/tmp/custorder.adm";
+select element {'cid':cid,'cust':(from g select c),'cnt-orders':count(o),'orders': (from g select o)}
+from  Customers as c,
+      Orders as o
+where (c.cid = o.cid)
+group by c.cid as cid group as g
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
deleted file mode 100644
index 285d27b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TestType as {
-  "id" : string,
-  "idx" : string,
-  "no-idx" : string
-};
-
-create dataset TestSet(TestType) primary key "id";
-create index TestSetIndex on TestSet(idx);
-
-delete $x from dataset TestSet where $x.no-idx = "one" or $x.no-idx = "two";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp
new file mode 100644
index 0000000..40064ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : string,
+  idx : string,
+  `no-idx` : string
+};
+
+create  dataset TestSet(TestType) primary key id;
+
+create  index TestSetIndex  on TestSet (idx) type btree;
+
+delete from TestSet
+ where `no-idx` = 'one' or `no-idx` = 'two';
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
deleted file mode 100644
index 0425ecf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : July 11 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type myDataType as open {
-  id: int32
-}
-
-create dataset myData(myDataType)
-  primary key id;
-
-insert into dataset myData (
-for $x in dataset myData
-return {
-    "id": $x.id + 1
-}
-);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp
new file mode 100644
index 0000000..5566eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.myDataType as
+{
+  id : integer
+};
+
+create  dataset myData(myDataType) primary key id;
+
+insert into myData
+select element {'id':(x.id + 1)}
+from  myData as x
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
deleted file mode 100644
index 261a357..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse fuzzyjoin if exists;
-create dataverse fuzzyjoin;
-use dataverse fuzzyjoin;
-
-create type TOKENSRANKEDADMType as closed {
-  token: int32,
-  rank: int32
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
-
-write output to asterix_nc1:'rttest/introhashpartitionmerge.adm';
-
-        for $token1 in dataset('TOKENSRANKEDADM')
-        for $token2 in
-            for $tokenRanked in dataset('TOKENSRANKEDADM')
-            order by $tokenRanked.rank
-            return $tokenRanked
-        where $token1.token = $token2.token
-        order by $token2.rank
-        return $token2.rank
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
new file mode 100644
index 0000000..103aae8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse fuzzyjoin if exists;
+create  dataverse fuzzyjoin;
+
+use fuzzyjoin;
+
+
+create type fuzzyjoin.TOKENSRANKEDADMType as
+ closed {
+  token : integer,
+  rank : integer
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
+
+write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
+select element token2.rank
+from  TOKENSRANKEDADM as token1,
+      (
+    select element tokenRanked
+    from  TOKENSRANKEDADM as tokenRanked
+    order by tokenRanked.rank
+) as token2
+where (token1.token = token2.token)
+order by token2.rank
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
deleted file mode 100644
index 488ed70..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
-
-// Cannot optimize this query because the string constant is shorter than the gram length.
-for $o in dataset('DBLP')
-where contains($o.title, "Mu")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
new file mode 100644
index 0000000..e13305d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
deleted file mode 100644
index ba460a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
new file mode 100644
index 0000000..e5b4885
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
deleted file mode 100644
index 81780a2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
-
-for $o in dataset('DBLP')
-where edit-distance-check($o.authors, "Amihay Motro", 1)[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
new file mode 100644
index 0000000..da55c99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
+select element o
+from  DBLP as o
+where test.`edit-distance-check`(o.authors,'Amihay Motro',1)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
deleted file mode 100644
index 6917878..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
-
-for $o in dataset('DBLP')
-where edit-distance($o.authors, "Amihay Motro") <= 1
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
new file mode 100644
index 0000000..91a809b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
+select element o
+from  DBLP as o
+where (test.`edit-distance`(o.authors,'Amihay Motro') <= 1)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index b4d342c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '1';
-
-for $o in dataset('DBLP')
-where $o.authors ~= "Amihay Motro"
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..d684f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `1`;
+
+select element o
+from  DBLP as o
+where (o.authors ~= 'Amihay Motro')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
deleted file mode 100644
index 55f1830..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $o in dataset('DBLP')
-where gram-tokens($o.title, 3, false) ~= gram-tokens("Transactions for Cooperative Environments", 3, false)
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..5adcda0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element o
+from  DBLP as o
+where (test.`gram-tokens`(o.title,3,false) ~= test.`gram-tokens`('Transactions for Cooperative Environments',3,false))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
deleted file mode 100644
index c7d273d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
new file mode 100644
index 0000000..5b564db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
+select element o
+from  DBLP as o
+where test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
deleted file mode 100644
index da87bc0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false)) >= 0.5f
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
new file mode 100644
index 0000000..3e024a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
+select element o
+from  DBLP as o
+where (test.`similarity-jaccard`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
deleted file mode 100644
index 88b0d58..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
new file mode 100644
index 0000000..41ceade
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
deleted file mode 100644
index d107b68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
-
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
new file mode 100644
index 0000000..0ca8b00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
deleted file mode 100644
index f4d5e14..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 3
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
new file mode 100644
index 0000000..0f2fcf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
deleted file mode 100644
index 687e55d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
-
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 1
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
new file mode 100644
index 0000000..a4114d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
deleted file mode 100644
index 8833eaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '1';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["computers", "wine", "walking"]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..1bfd5f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `1`;
+
+select element c
+from  Customers as c
+where (c.interests ~= ['computers','wine','walking'])
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
deleted file mode 100644
index d0df75f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["databases", "computers", "wine"]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..a1372d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element c
+from  Customers as c
+where (c.interests ~= ['databases','computers','wine'])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
deleted file mode 100644
index 97e9962..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
new file mode 100644
index 0000000..89a3eb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
+select element c
+from  Customers as c
+where test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
deleted file mode 100644
index 329b26a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, ["databases", "computers", "wine"]) >= 0.7f
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
new file mode 100644
index 0000000..d13c98b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
+select element c
+from  Customers as c
+where (test.`similarity-jaccard`(c.interests,['databases','computers','wine']) >= 0.700000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
deleted file mode 100644
index c20493a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $c in dataset('Customers')
-where $c.interests ~= {{"computers", "wine", "databases"}}
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..d32a8f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element c
+from  Customers as c
+where (c.interests ~= {{'computers','wine','databases'}})
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
deleted file mode 100644
index cb6c0c4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, {{"computers", "wine", "databases"}}, 0.7f)[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
new file mode 100644
index 0000000..25568e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+select element c
+from  Customers as c
+where test.`similarity-jaccard-check`(c.interests,{{'computers','wine','databases'}},0.700000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
deleted file mode 100644
index c0cdd4f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, {{"computers", "databases", "wine"}}) >= 0.7f
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
new file mode 100644
index 0000000..2a26ee9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+select element c
+from  Customers as c
+where (test.`similarity-jaccard`(c.interests,{{'computers','databases','wine'}}) >= 0.700000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
deleted file mode 100644
index 191a9bf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
-
-// Contains cannot be answered with a word inverted index.
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
new file mode 100644
index 0000000..2a1db6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
deleted file mode 100644
index ad2c577..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $o in dataset('DBLP')
-where word-tokens($o.title) ~= word-tokens("Transactions for Cooperative Environments")
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..6157fde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element o
+from  DBLP as o
+where (test.`word-tokens`(o.title) ~= test.`word-tokens`('Transactions for Cooperative Environments'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
deleted file mode 100644
index 6f4497f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)[0]
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
new file mode 100644
index 0000000..b0dc05a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
+select element o
+from  DBLP as o
+where test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
deleted file mode 100644
index c96de32..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments")) >= 0.5f
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
new file mode 100644
index 0000000..eb96ddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
+select element o
+from  DBLP as o
+where (test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
deleted file mode 100644
index cbe5b17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
- *                  two edit-distance-check function of which only the first can be optimized with an index. 
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
-
-// Only the first edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-where $eda[0] and $edb[0] 
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
new file mode 100644
index 0000000..c60c8fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using
+ *                  two edit-distance-check function of which only the first can be optimized with an index.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+select element o
+from  DBLP as o
+with  eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3),
+      edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
+where (eda[0] and edb[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
deleted file mode 100644
index a40a4de..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
- *                  two edit-distance-check function of which only the second can be optimized with an index.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
-
-// Only the second edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-where $edb[0] and $eda[0] 
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
new file mode 100644
index 0000000..8a5d1d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using
+ *                  two edit-distance-check function of which only the second can be optimized with an index.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+select element o
+from  DBLP as o
+with  edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5),
+      eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3)
+where (edb[0] and eda[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
deleted file mode 100644
index 09173ce..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
-
-for $o in dataset('DBLP')
-let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
-where $ed[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
new file mode 100644
index 0000000..1f725d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
+select element o
+from  DBLP as o
+with  ed as test.`edit-distance-check`(o.authors,'Amihay Motro',1)
+where ed[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
deleted file mode 100644
index 73293bc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
- *                  Tests that the optimizer rule correctly drills through the substring function.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
-
-for $paper in dataset('DBLP')
-where edit-distance-check(substring($paper.title, 0, 8), "datbase", 1)[0]
-return {
-  "id" : $paper.id,
-  "title" : $paper.title
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
new file mode 100644
index 0000000..d2e3888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
+ *                  Tests that the optimizer rule correctly drills through the substring function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
+select element {'id':paper.id,'title':paper.title}
+from  DBLP as paper
+where test.`edit-distance-check`(test.substring(paper.title,0,8),'datbase',1)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
deleted file mode 100644
index 3271c8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
-where $jacc[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..248c243
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
+select element o
+from  DBLP as o
+with  jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
deleted file mode 100644
index dfeda68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := gram-tokens($paper.title, 3, false)
-let $query_tokens := gram-tokens("Transactions for Cooperative Environments", 3, false)
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
new file mode 100644
index 0000000..8be4caa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
+select element {'Paper':paper_tokens,'Query':query_tokens}
+from  DBLP as paper
+with  paper_tokens as test.`gram-tokens`(paper.title,3,false),
+      query_tokens as test.`gram-tokens`('Transactions for Cooperative Environments',3,false),
+      jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
deleted file mode 100644
index 2cb65d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
new file mode 100644
index 0000000..68106a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
deleted file mode 100644
index 9646383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
new file mode 100644
index 0000000..7083657
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
deleted file mode 100644
index a274b9f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..a04a8ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
+select element c
+from  Customers as c
+with  jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
deleted file mode 100644
index 9d3ca6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..ceac8c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
+select element c
+from  Customers as c
+with  jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
deleted file mode 100644
index d932d23..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..e4f3b8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
+select element o
+from  DBLP as o
+with  jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
deleted file mode 100644
index 79fe390..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := word-tokens($paper.title)
-let $query_tokens := word-tokens("Transactions for Cooperative Environments")
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.8f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
new file mode 100644
index 0000000..56ede6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
+select element {'Paper':paper_tokens,'Query':query_tokens}
+from  DBLP as paper
+with  paper_tokens as test.`word-tokens`(paper.title),
+      query_tokens as test.`word-tokens`('Transactions for Cooperative Environments'),
+      jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.800000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
deleted file mode 100644
index b99bc22..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
new file mode 100644
index 0000000..c29de43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
+select element {'aauthors':a.authors,'bauthors':b.authors,'ed':ed}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index d4491eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary. 
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..1234824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'aauthors':a.authors,'bauthors':b.authors}
+from  DBLP as a,
+      CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
deleted file mode 100644
index 04d57a4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
new file mode 100644
index 0000000..4fdc637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
+select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
deleted file mode 100644
index 5d48e10..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
new file mode 100644
index 0000000..40c8c5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'ed':ed}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
deleted file mode 100644
index d074b0f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
new file mode 100644
index 0000000..a003dee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
deleted file mode 100644
index 56697c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
new file mode 100644
index 0000000..1372490
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
deleted file mode 100644
index b917cda..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
new file mode 100644
index 0000000..9530163
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
+select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
deleted file mode 100644
index 8ddd03b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Test that left-outer-join may use an available inverted index in index subtree.
- * Issue        : 741
- * Expected Res : Success
- * Date         : 16th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as {
-screen_name: string,
-lang: string,
-friends_count: int32,
-statuses_count: int32,
-name: string,
-followers_count: int32
-}
-
-create type TweetMessageType as {
-tweetid: int64,
-user: TwitterUserType,
-sender_location: point?,
-send_time: datetime,
-referred_topics: {{ string }},
-message_text: string
-}
-
-create dataset TweetMessages(TweetMessageType) primary key tweetid;
-
-create index topicIIx on TweetMessages(referred_topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
-
-for $t in dataset('TweetMessages')
-where $t.send_time >= datetime('2011-06-18T14:10:17')
-and
-$t.send_time < datetime('2011-06-18T15:10:17')
-return {
-    "tweet": $t.tweetid,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := similarity-jaccard-check($t.referred_topics, $t2.referred_topics, 0.6f)
-              where $sim[0] and
-                      $t2.tweetid != $t.tweetid
-                      return $t2.tweetid
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
new file mode 100644
index 0000000..5a312b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use an available inverted index in index subtree.
+ * Issue        : 741
+ * Expected Res : Success
+ * Date         : 16th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+{
+  screen_name : string,
+  lang : string,
+  friends_count : integer,
+  statuses_count : integer,
+  name : string,
+  followers_count : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  sender_location : point?,
+  send_time : datetime,
+  referred_topics : {{string}},
+  message_text : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index topicIIx  on TweetMessages (referred_topics) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
+select element {'tweet':t.tweetid,'similar-tweets':(
+        select element t2.tweetid
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t.referred_topics,t2.referred_topics,0.600000f)
+        where (sim[0] and (t2.tweetid != t.tweetid))
+    )}
+from  TweetMessages as t
+where ((t.send_time >= test.datetime('2011-06-18T14:10:17')) and (t.send_time < test.datetime('2011-06-18T15:10:17')))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
deleted file mode 100644
index bd52bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-create index topicKeywordIx on TweetMessages(referred-topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
-    "tweet": {"id": $t1.tweetid, "topics" : $t1.message-text} ,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := edit-distance-check($t1.message-text, $t2.message-text, 7)
-              where $sim[0] and
-                      $t2.tweetid != $t1.tweetid
-                      order by $t2.tweetid
-                      return {"id": $t2.tweetid, "topics" : $t2.message-text}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
new file mode 100644
index 0000000..8c2869f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
deleted file mode 100644
index 27b0e37..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-create index topicKeywordIx on TweetMessages(referred-topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
-    "tweet": {"id": $t1.tweetid, "topics" : $t1.referred-topics} ,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := similarity-jaccard-check($t1.referred-topics, $t2.referred-topics, 0.5f)
-              where $sim[0] and
-                      $t2.tweetid != $t1.tweetid
-                      order by $t2.tweetid
-                      return {"id": $t2.tweetid, "topics" : $t2.referred-topics}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
new file mode 100644
index 0000000..9db0bd7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.`referred-topics`,t2.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
deleted file mode 100644
index cf2222d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
- *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
new file mode 100644
index 0000000..a6ade32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
deleted file mode 100644
index 95b2912..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32,
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
new file mode 100644
index 0000000..ed8d6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
deleted file mode 100644
index 5961d13..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance-check($a.authors, $b.authors, 3)
-where $ed[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
new file mode 100644
index 0000000..7a24772
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
+select element {'arec':a,'brec':b,'ed':ed[1]}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance-check`(a.authors,b.authors,3)
+where (ed[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
deleted file mode 100644
index c1a1817..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
- *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
new file mode 100644
index 0000000..950a058
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
deleted file mode 100644
index c65b61b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
new file mode 100644
index 0000000..dc8ca3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
deleted file mode 100644
index 6287cb0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index. 
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
new file mode 100644
index 0000000..0284ff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
+select element {'arec':a,'brec':b,'ed':ed}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
deleted file mode 100644
index 66bb2ac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
new file mode 100644
index 0000000..2021605
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c611d3a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
new file mode 100644
index 0000000..d900a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 085a383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..aac9ca7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 444b735..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..d3bcd49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
deleted file mode 100644
index 31ec7a5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..460f1cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
deleted file mode 100644
index 6255558..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..5e180f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
deleted file mode 100644
index f70df08..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..2b9aa31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
deleted file mode 100644
index 2e1b065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
new file mode 100644
index 0000000..a2dc8f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
deleted file mode 100644
index 31003d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
new file mode 100644
index 0000000..fbc230c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
deleted file mode 100644
index 1e86763..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
new file mode 100644
index 0000000..5265874
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
deleted file mode 100644
index f074ad1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
new file mode 100644
index 0000000..f7e6047
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
deleted file mode 100644
index 723d0b5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
new file mode 100644
index 0000000..6435990
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
deleted file mode 100644
index 0bb95d5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance-check($a.interests, $b.interests, 3)
-where $ed[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
new file mode 100644
index 0000000..9f14368
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
+select element {'arec':a,'brec':b,'ed':ed[1]}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance-check`(a.interests,b.interests,3)
+where (ed[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
deleted file mode 100644
index e5f5b40..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
new file mode 100644
index 0000000..5ebb518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
deleted file mode 100644
index f78a6bd..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
new file mode 100644
index 0000000..ac7e6ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
deleted file mode 100644
index c4e037f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
new file mode 100644
index 0000000..0655746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
+select element {'arec':a,'brec':b,'ed':ed}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
deleted file mode 100644
index c5a558c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
new file mode 100644
index 0000000..cd3d74f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c96c0e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
new file mode 100644
index 0000000..a184602
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index f90593a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..c1f3af7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 264c93a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..573f5e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
deleted file mode 100644
index ce60fe5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..c9cbd1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
deleted file mode 100644
index edaba17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..6364474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
deleted file mode 100644
index 3d704f2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..bd84396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
deleted file mode 100644
index 55c34ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
new file mode 100644
index 0000000..b6bea6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
deleted file mode 100644
index 37725d9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
new file mode 100644
index 0000000..3359746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
deleted file mode 100644
index 1bba03c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
new file mode 100644
index 0000000..941d506
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index b918bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..5c46ddc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 64260d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..b874be2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
deleted file mode 100644
index 0a1f2f8..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..2a771e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
deleted file mode 100644
index f489f34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..8d71bb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
deleted file mode 100644
index 6c87995..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..476db19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
deleted file mode 100644
index 8a6d743..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
new file mode 100644
index 0000000..287ef06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
deleted file mode 100644
index ef5269a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
new file mode 100644
index 0000000..bc7b1cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
deleted file mode 100644
index 72a8dca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
new file mode 100644
index 0000000..77c26cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 6b1feb6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..e2158e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index d305ba0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..c6c38b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index 5586919..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- *                  TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- *                    transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- *                    multiple indexed nested loop joins of various type of indexes.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-    user: TwitterUserType,
-    sender-location: point,
-    send-time: datetime,
-    referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.message-text), word-tokens($t2.message-text), 0.6f)
-where $sim[0] and $t1.tweetid < int64("20") and $t2.tweetid != $t1.tweetid
-return {
-    "t1": $t1.tweetid,
-    "t2": $t2.tweetid,
-    "sim": $sim[1]
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
new file mode 100644
index 0000000..50d2442
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
+ *                  TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
+ *                    transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
+ *                    multiple indexed nested loop joins of various type of indexes.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
+select element {'t1':t1.tweetid,'t2':t2.tweetid,'sim':sim[1]}
+from  TweetMessages as t1,
+      TweetMessages as t2
+with  sim as test.`similarity-jaccard-check`(test.`word-tokens`(t1.`message-text`),test.`word-tokens`(t2.`message-text`),0.600000f)
+where (sim[0] and (t1.tweetid < test.bigint('20')) and (t2.tweetid != t1.tweetid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
deleted file mode 100644
index 0a63d12..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..0c2890a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
deleted file mode 100644
index 5c97de1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..fdf6235
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
deleted file mode 100644
index 7481ed3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..8eb16d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
deleted file mode 100644
index 1b123a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
new file mode 100644
index 0000000..b943eec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
deleted file mode 100644
index e0a2568..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
new file mode 100644
index 0000000..37a119b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
deleted file mode 100644
index f3ca957..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
new file mode 100644
index 0000000..584522a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
deleted file mode 100644
index 65029f3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse join-super-key_1 if exists;
-
-create dataverse join-super-key_1;
-
-use dataverse join-super-key_1;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $li in dataset('LineItems')
-for $ps in dataset('PartSupp')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and 
-      $li.l_extendedprice = $ps.ps_supplycost
-return {
-  "l_partkey": $li.l_partkey
-}
-
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
new file mode 100644
index 0000000..0e7fc0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `join-super-key_1` if exists;
+create  dataverse `join-super-key_1`;
+
+use `join-super-key_1`;
+
+
+create type `join-super-key_1`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `join-super-key_1`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `join-super-key_1`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `join-super-key_1`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `join-super-key_1`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/join-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from  LineItems as li,
+      PartSupp as ps
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
deleted file mode 100644
index 831976f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse join-super-key_01 if exists;
-
-create dataverse join-super-key_01;
-
-use dataverse join-super-key_01;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $ps in dataset('PartSupp')
-for $li in dataset('LineItems')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and 
-      $li.l_extendedprice = $ps.ps_supplycost
-return {
-  "l_partkey": $li.l_partkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
new file mode 100644
index 0000000..7247cdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `join-super-key_01` if exists;
+create  dataverse `join-super-key_01`;
+
+use `join-super-key_01`;
+
+
+create type `join-super-key_01`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `join-super-key_01`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `join-super-key_01`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `join-super-key_01`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `join-super-key_01`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/join-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from  PartSupp as ps,
+      LineItems as li
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
deleted file mode 100644
index ad6833d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse loj-super-key_01 if exists;
-
-create dataverse loj-super-key_01;
-
-use dataverse loj-super-key_01;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $li in dataset('LineItems')
-let $partsupp := 
-   for $ps in dataset('PartSupp')
-   where $li.l_partkey = $ps.ps_partkey 
-         and $li.l_suppkey = $ps.ps_suppkey  
-         and $li.l_extendedprice = $ps.ps_supplycost
-   return $ps
-return {
-  "li": $li,
-  "partsupp": $partsupp 
-} 
- 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
new file mode 100644
index 0000000..38bd3e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `loj-super-key_01` if exists;
+create  dataverse `loj-super-key_01`;
+
+use `loj-super-key_01`;
+
+
+create type `loj-super-key_01`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `loj-super-key_01`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `loj-super-key_01`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `loj-super-key_01`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `loj-super-key_01`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'li':li,'partsupp':partsupp}
+from  LineItems as li
+with  partsupp as (
+      select element ps
+      from  PartSupp as ps
+      where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
deleted file mode 100644
index e0678e9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse loj-super-key_02 if exists;
-
-create dataverse loj-super-key_02;
-
-use dataverse loj-super-key_02;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $ps in dataset('PartSupp')
-let $items := 
-   for $li in dataset('LineItems')   
-   where $li.l_partkey = $ps.ps_partkey 
-         and $li.l_suppkey = $ps.ps_suppkey  
-         and $li.l_extendedprice = $ps.ps_supplycost
-   return $li
-return {
-  "partsupp": $ps,
-  "li": $items 
-} 
- 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
new file mode 100644
index 0000000..9705785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `loj-super-key_02` if exists;
+create  dataverse `loj-super-key_02`;
+
+use `loj-super-key_02`;
+
+
+create type `loj-super-key_02`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `loj-super-key_02`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `loj-super-key_02`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `loj-super-key_02`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `loj-super-key_02`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'partsupp':ps,'li':items}
+from  PartSupp as ps
+with  items as (
+      select element li
+      from  LineItems as li
+      where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
deleted file mode 100644
index f7504ad..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset LineItems(LineItemType)
-  primary key l_orderkey, l_linenumber on group1;
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-create dataset Customers(CustomerType) 
-  primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/nested_loj.adm";
-
-for $c in dataset('Customers')
-let $orders := 
-  for $o in dataset('Orders')
-  where $o.o_custkey = $c.c_custkey
-  let $items := 
-    for $l in dataset('LineItems')
-    where $l.l_orderkey = $o.o_orderkey
-    return $l
-  return {
-     "order": $o,
-     "items": $items
-  }  
-return {
-  "cust": $c,
-  "orders": $orders
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
new file mode 100644
index 0000000..aae756c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset LineItems(LineItemType) primary key l_orderkey,l_linenumber on group1;
+
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+create  dataset Customers(CustomerType) primary key c_custkey on group1;
+
+write output to asterix_nc1:"/tmp/nested_loj.adm";
+select element {'cust':c,'orders':orders}
+from  Customers as c
+with  orders as (
+      select element {'order':o,'items':items}
+      from  Orders as o
+      with  items as (
+            select element l
+            from  LineItems as l
+            where (l.l_orderkey = o.o_orderkey)
+        )
+      where (o.o_custkey = c.c_custkey)
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
deleted file mode 100644
index 544e715..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
new file mode 100644
index 0000000..190f2f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
deleted file mode 100644
index 3f9aa3f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey,
-  "o_totalprice": $o.o_totalprice
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
new file mode 100644
index 0000000..64e2395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
deleted file mode 100644
index 4eaf437..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
new file mode 100644
index 0000000..4fe9e18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
deleted file mode 100644
index b2adee0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey,
-  "o_totalprice": $o.o_totalprice
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
new file mode 100644
index 0000000..c16810f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
deleted file mode 100644
index a909b11..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse pull-select-above-eq-join if exists;
-
-create dataverse pull-select-above-eq-join;
-
-use dataverse pull-select-above-eq-join;
-
-create type UserType as open {
-  uid: int32, 
-  name: string,
-  city: string,
-  lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
-  vid: int32, 
-  name: string,
-  city: string,
-  lottery_numbers: {{int32}}
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Users(UserType) 
-  primary key uid on group1;
-
-create dataset Visitors(VisitorType) 
-  primary key vid on group1;
-
-
-write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.name = $visitor.name
-      and $user.city != $visitor.city
-return {"uid": $user.uid, "vid": $visitor.vid}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
new file mode 100644
index 0000000..8634de5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `pull-select-above-eq-join` if exists;
+create  dataverse `pull-select-above-eq-join`;
+
+use `pull-select-above-eq-join`;
+
+
+create type `pull-select-above-eq-join`.UserType as
+{
+  uid : integer,
+  name : string,
+  city : string,
+  lottery_numbers : {{integer}}
+};
+
+create type `pull-select-above-eq-join`.VisitorType as
+{
+  vid : integer,
+  name : string,
+  city : string,
+  lottery_numbers : {{integer}}
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Users(UserType) primary key uid on group1;
+
+create  dataset Visitors(VisitorType) primary key vid on group1;
+
+write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
+select element {'uid':user.uid,'vid':visitor.vid}
+from  Users as user,
+      Visitors as visitor
+where ((user.name = visitor.name) and (user.city != visitor.city))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
deleted file mode 100644
index 4b4ed34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse fuzzyjoin_080 if exists;
-
-create dataverse fuzzyjoin_080;
-
-use dataverse fuzzyjoin_080;
-
-create type DBLPType as open {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-
-write output to asterix_nc1:'rttest/fuzzyjoin_080.adm';
-
-    for $paperDBLP in dataset('DBLP')
-    let $matches :=
-            for $paper in dataset('DBLP')
-            where $paper.authors = $paperDBLP.authors        
-            return $paper.title
-    return {'id': $paperDBLP.id, 'matches':$matches}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
new file mode 100644
index 0000000..535ae58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse fuzzyjoin_080 if exists;
+create  dataverse fuzzyjoin_080;
+
+use fuzzyjoin_080;
+
+
+create type fuzzyjoin_080.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset DBLP(DBLPType) primary key id on group1;
+
+write output to asterix_nc1:"rttest/fuzzyjoin_080.adm";
+select element {'id':paperDBLP.id,'matches':matches}
+from  DBLP as paperDBLP
+with  matches as (
+      select element paper.title
+      from  DBLP as paper
+      where (paper.authors = paperDBLP.authors)
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
deleted file mode 100644
index 0961e44..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse q08_group_by if exists;
-
-create dataverse q08_group_by;
-
-use dataverse q08_group_by;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type RegionType as closed {
-  r_regionkey: int32,
-  r_name: string,
-  r_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
-  primary key o_orderkey;
-create dataset Customer(CustomerType) 
-  primary key c_custkey;
-create dataset Supplier(SupplierType)
-  primary key s_suppkey;
-create dataset Nation(NationType) 
-  primary key n_nationkey;
-create dataset Region(RegionType)
-  primary key r_regionkey;
-create dataset Part(PartType)
-  primary key p_partkey;
-
-for $s in dataset("Supplier")
-    for $lnrcop in (
-      for $lnrco in (
-        for $l in dataset('LineItem')
-        for $nrco in (
-          for $o in dataset('Orders')
-          for $nrc in (
-            for $c in dataset('Customer')
-            for $nr in (
-              for $n1 in dataset('Nation')
-              for $r1 in dataset('Region')
-              where $n1.n_regionkey = $r1.r_regionkey and $r1.r_name = 'AMERICA'
-              return { "n_nationkey": $n1.n_nationkey }
-            )
-            where $c.c_nationkey = $nr.n_nationkey
-            return { "c_custkey": $c.c_custkey }
-          )
-          where $nrc.c_custkey = $o.o_custkey
-          return {
-            "o_orderdate" : $o.o_orderdate, 
-            "o_orderkey": $o.o_orderkey 
-          }
-        )
-        where $l.l_orderkey = $nrco.o_orderkey
-          and $nrco.o_orderdate >= '1995-01-01' 
-          and $nrco.o_orderdate <= '1996-12-31'
-        return {
-          "o_orderdate": $nrco.o_orderdate, 
-          "l_partkey": $l.l_partkey, 
-          "l_discount": $l.l_discount, 
-          "l_extendedprice": $l.l_extendedprice, 
-          "l_suppkey": $l.l_suppkey
-        }
-      )
-      for $p in dataset("Part")
-      where $p.p_partkey = $lnrco.l_partkey and $p.p_type = 'ECONOMY ANODIZED STEEL'
-      return {
-        "o_orderdate": $lnrco.o_orderdate, 
-        "l_discount": $lnrco.l_discount, 
-        "l_extendedprice": $lnrco.l_extendedprice, 
-        "l_suppkey": $lnrco.l_suppkey 
-      }
-    )
-    where $s.s_suppkey = $lnrcop.l_suppkey
-    return {
-      "o_orderdate": $lnrcop.o_orderdate, 
-      "l_discount": $lnrcop.l_discount, 
-      "l_extendedprice": $lnrcop.l_extendedprice, 
-      "l_suppkey": $lnrcop.l_suppkey, 
-      "s_nationkey": $s.s_nationkey
-    }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
new file mode 100644
index 0000000..4ff3d31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse q08_group_by if exists;
+create  dataverse q08_group_by;
+
+use q08_group_by;
+
+
+create type q08_group_by.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type q08_group_by.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type q08_group_by.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create type q08_group_by.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type q08_group_by.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type q08_group_by.RegionType as
+ closed {
+  r_regionkey : integer,
+  r_name : string,
+  r_comment : string
+};
+
+create type q08_group_by.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+select element {'o_orderdate':lnrcop.o_orderdate,'l_discount':lnrcop.l_discount,'l_extendedprice':lnrcop.l_extendedprice,'l_suppkey':lnrcop.l_suppkey,'s_nationkey':s.s_nationkey}
+from  Supplier as s,
+      (
+    select element {'o_orderdate':lnrco.o_orderdate,'l_discount':lnrco.l_discount,'l_extendedprice':lnrco.l_extendedprice,'l_suppkey':lnrco.l_suppkey}
+    from  (
+        select element {'o_orderdate':nrco.o_orderdate,'l_partkey':l.l_partkey,'l_discount':l.l_discount,'l_extendedprice':l.l_extendedprice,'l_suppkey':l.l_suppkey}
+        from  LineItem as l,
+              (
+            select element {'o_orderdate':o.o_orderdate,'o_orderkey':o.o_orderkey}
+            from  Orders as o,
+                  (
+                select element {'c_custkey':c.c_custkey}
+                from  Customer as c,
+                      (
+                    select element {'n_nationkey':n1.n_nationkey}
+                    from  Nation as n1,
+                          Region as r1
+                    where ((n1.n_regionkey = r1.r_regionkey) and (r1.r_name = 'AMERICA'))
+                ) as nr
+                where (c.c_nationkey = nr.n_nationkey)
+            ) as nrc
+            where (nrc.c_custkey = o.o_custkey)
+        ) as nrco
+        where ((l.l_orderkey = nrco.o_orderkey) and (nrco.o_orderdate >= '1995-01-01') and (nrco.o_orderdate <= '1996-12-31'))
+    ) as lnrco,
+          Part as p
+    where ((p.p_partkey = lnrco.l_partkey) and (p.p_type = 'ECONOMY ANODIZED STEEL'))
+) as lnrcop
+where (s.s_suppkey = lnrcop.l_suppkey)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
deleted file mode 100644
index adb6efe..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse q09_group_by if exists;
-
-create dataverse q09_group_by;
-
-use dataverse q09_group_by;
-
-create type LineItemType as closed {
-  l_orderkey: int32,
-  l_partkey: int32,
-  l_suppkey: int32,
-  l_linenumber: int32,
-  l_quantity: int32,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32,
-  o_custkey: int32,
-  o_orderstatus: string,
-  o_totalprice: double,
-  o_orderdate: string,
-  o_orderpriority: string,
-  o_clerk: string,
-  o_shippriority: int32,
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32,
-  c_name: string,
-  c_address: string,
-  c_nationkey: int32,
-  c_phone: string,
-  c_acctbal: double,
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create type SupplierType as closed {
-  s_suppkey: int32,
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type RegionType as closed {
-    r_regionkey: int32,
-    r_name: string,
-    r_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32,
-  p_name: string,
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32,
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
-  primary key o_orderkey;
-create dataset Supplier(SupplierType)
-  primary key s_suppkey;
-create dataset Region(RegionType)
-  primary key r_regionkey;
-create dataset Nation(NationType)
-  primary key n_nationkey;
-create dataset Part(PartType)
-  primary key p_partkey;
-create dataset Partsupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey;
-create dataset Customer(CustomerType)
-  primary key c_custkey;
-
-for $p in dataset('Part')
-    for $l2 in (
-      for $ps in dataset('Partsupp')
-      for $l1 in (
-        for $s1 in (
-          for $s in dataset('Supplier')
-          for $n in dataset('Nation')
-          where $n.n_nationkey = $s.s_nationkey
-          return {
-            "s_suppkey": $s.s_suppkey,
-            "n_name": $n.n_name
-          }
-        )
-        for $l in dataset('LineItem')
-        where $s1.s_suppkey = $l.l_suppkey
-        return  {
-          "l_suppkey": $l.l_suppkey,
-          "l_extendedprice": $l.l_extendedprice,
-          "l_discount": $l.l_discount,
-          "l_quantity": $l.l_quantity,
-          "l_partkey": $l.l_partkey,
-          "l_orderkey": $l.l_orderkey,
-          "n_name": $s1.n_name
-        }
-      )
-      where $ps.ps_suppkey = $l1.l_suppkey and $ps.ps_partkey = $l1.l_partkey
-      return {
-        "l_extendedprice": $l1.l_extendedprice,
-        "l_discount": $l1.l_discount,
-        "l_quantity": $l1.l_quantity,
-        "l_partkey": $l1.l_partkey,
-        "l_orderkey": $l1.l_orderkey,
-        "n_name": $l1.n_name,
-        "ps_supplycost": $ps.ps_supplycost
-      }
-    )
-    where contains($p.p_name, 'green') and $p.p_partkey = $l2.l_partkey
-    return {
-      "l_extendedprice": $l2.l_extendedprice,
-      "l_discount": $l2.l_discount,
-      "l_quantity": $l2.l_quantity,
-      "l_orderkey": $l2.l_orderkey,
-      "n_name": $l2.n_name,
-      "ps_supplycost": $l2.ps_supplycost
-    }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
new file mode 100644
index 0000000..44f4760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse q09_group_by if exists;
+create  dataverse q09_group_by;
+
+use q09_group_by;
+
+
+create type q09_group_by.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : integer,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type q09_group_by.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type q09_group_by.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create type q09_group_by.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type q09_group_by.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type q09_group_by.RegionType as
+ closed {
+  r_regionkey : integer,
+  r_name : string,
+  r_comment : string
+};
+
+create type q09_group_by.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type q09_group_by.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+create  dataset Partsupp(PartSuppType) primary key ps_partkey,ps_suppkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select element {'l_extendedprice':l2.l_extendedprice,'l_discount':l2.l_discount,'l_quantity':l2.l_quantity,'l_orderkey':l2.l_orderkey,'n_name':l2.n_name,'ps_supplycost':l2.ps_supplycost}
+from  Part as p,
+      (
+    select element {'l_extendedprice':l1.l_extendedprice,'l_discount':l1.l_discount,'l_quantity':l1.l_quantity,'l_partkey':l1.l_partkey,'l_orderkey':l1.l_orderkey,'n_name':l1.n_name,'ps_supplycost':ps.ps_supplycost}
+    from  Partsupp as ps,
+          (
+        select element {'l_suppkey':l.l_suppkey,'l_extendedprice':l.l_extendedprice,'l_discount':l.l_discount,'l_quantity':l.l_quantity,'l_partkey':l.l_partkey,'l_orderkey':l.l_orderkey,'n_name':s1.n_name}
+        from  (
+            select element {'s_suppkey':s.s_suppkey,'n_name':n.n_name}
+            from  Supplier as s,
+                  Nation as n
+            where (n.n_nationkey = s.s_nationkey)
+        ) as s1,
+              LineItem as l
+        where (s1.s_suppkey = l.l_suppkey)
+    ) as l1
+    where ((ps.ps_suppkey = l1.l_suppkey) and (ps.ps_partkey = l1.l_partkey))
+) as l2
+where (q09_group_by.contains(p.p_name,'green') and (p.p_partkey = l2.l_partkey))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
index 8da16a0..a46d514 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
@@ -32,7 +32,7 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 LEFT OUTER JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))>2 AND STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id DESC, fu2.id DESC
 LIMIT 5;
 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
index a05087b..942f39b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
index 4b94bf6..7dbbab9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
index 0a46e9a..4084992 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
deleted file mode 100644
index 827b02a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : This test case is to verify the fix for issue601
- * https://code.google.com/p/asterixdb/issues/detail?id=601
- * Expected Res : SUCCESS
- * Date         : 10th Oct 2014
- */
-
-drop dataverse tpch if exists;
-create dataverse tpch;
-
-use dataverse tpch;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-  
-for $l in dataset('LineItem')
-group by $l_linenumber := $l.l_linenumber with $l
-return {
-  "l_linenumber": $l_linenumber,
-  "count_order": count($l)
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
new file mode 100644
index 0000000..a5861be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue601
+ * https://code.google.com/p/asterixdb/issues/detail?id=601
+ * Expected Res : SUCCESS
+ * Date         : 10th Oct 2014
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+select element {'l_linenumber':l_linenumber,'count_order':count(l)}
+from  LineItem as l
+group by l.l_linenumber as l_linenumber
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
deleted file mode 100644
index 18acdf0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date         : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in dataset s
-for $y in (
- for $z in {{ {"a":1, "c":1},{"a":2, "c":2},{"a":1, "c":null} }} where $x.b=$z.a
-    return $z.c
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp
new file mode 100644
index 0000000..ee21bc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date         : 2nd Feb. 2015
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+  b : integer
+};
+
+create  dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from  s as x,
+      (
+    select element z.c
+    from  {{{'a':1,'c':1},{'a':2,'c':2},{'a':1,'c':null}}} as z
+    where (x.b = z.a)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
deleted file mode 100644
index 3e2bea3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date         : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in {{ {"a":1},{"a":2} }}
-for $y in (
- for $z in dataset s where $x.a=$z.b
-    return $z.b
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp
new file mode 100644
index 0000000..5b0e7fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date         : 2nd Feb. 2015
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+  b : integer
+};
+
+create  dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from  {{{'a':1},{'a':2}}} as x,
+      (
+    select element z.b
+    from  s as z
+    where (x.a = z.b)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
deleted file mode 100644
index 2b6df8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
-
-for $t1 in dataset('TweetMessages')
-let $n :=  create-circle($t1.sender-location, 0.5)
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"loc1":$t1.sender-location,
-"nearby-message": for $t2 in dataset('TweetMessages')
-                             where spatial-intersect($t2.sender-location, $n) and $t1.tweetid != $t2.tweetid
-                             order by $t2.tweetid
-                             return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
new file mode 100644
index 0000000..a30f22c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
deleted file mode 100644
index dcbf53f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData2' has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData1(MyRecord) primary key id;
-create dataset MyData2(MyRecord) primary key id;
-
-create index rtree_index on MyData2(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
-
-for $a in dataset('MyData1')
-for $b in dataset('MyData2')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
new file mode 100644
index 0000000..f87832e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+  id : integer,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create  dataset MyData1(MyRecord) primary key id;
+
+create  dataset MyData2(MyRecord) primary key id;
+
+create  index rtree_index  on MyData2 (point) type rtree;
+
+write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
+select element {'a':a,'b':b}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
deleted file mode 100644
index 624fe9e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description    : Self-joins a dataset on the intersection of its point attribute.
- *                  The dataset has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData(MyRecord) primary key id;
-
-create index rtree_index on MyData(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
-
-for $a in dataset('MyData')
-for $b in dataset('MyData')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
new file mode 100644
index 0000000..afe7a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Self-joins a dataset on the intersection of its point attribute.
+ *                  The dataset has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+  id : integer,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create  dataset MyData(MyRecord) primary key id;
+
+create  index rtree_index  on MyData (point) type rtree;
+
+write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
+select element {'a':a,'b':b}
+from  MyData as a,
+      MyData as b
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
deleted file mode 100644
index b8c7a8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
- * Expected Res : Success
- * Date         : 21th December 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname /*+ skip-index */ >= "Max" and $emp.fname <= "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
new file mode 100644
index 0000000..6235842
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
+ * Expected Res : Success
+ * Date         : 21th December 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname /*+ skip-index */  >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
deleted file mode 100644
index a3bb0fc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-
-create type EmploymentType as open {
-        organization-name: string,      
-        start-date: date,
-        end-date: date?
-}
-
-create type FacebookUserType as closed {
-        id: int32,
-        alias: string,
-        name: string,
-        user-since: datetime,
-        friend-ids: {{ int32 }},
-        employment: [EmploymentType]
-}
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-
-let $lonelyusers := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-let $lonelyusers2 := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-for $l1 in $lonelyusers
-for $l2 in $lonelyusers2
-where $l1.id < $l2.id
-order by $l1.id, $l2.id
-return { "user1": { "id": $l1.id, "name": $l1.name }, "user2": { "id": $l2.id, "name": $l2.name } };
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
new file mode 100644
index 0000000..c77559a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.EmploymentType as
+{
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type TinySocial.FacebookUserType as
+ closed {
+  id : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
+with  lonelyusers as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  ),
+      lonelyusers2 as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  )
+select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
+from  lonelyusers as l1,
+      lonelyusers2 as l2
+where (l1.id < l2.id)
+order by l1.id,l2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
index 7819959..8d97b03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
@@ -79,14 +79,14 @@
 create  dataset Customers(CustomerType) primary key c_custkey on group1;
 
 write output to asterix_nc1:"/tmp/inlined_q18_large_volume_customer.adm";
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':inlined_q18_large_volume_customer.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':inlined_q18_large_volume_customer.strict_sum((
         select element j.l_quantity
         from  l as j
     ))}
 from  Customers as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':inlined_q18_large_volume_customer.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':inlined_q18_large_volume_customer.strict_sum((
             select element i.l_quantity
             from  l as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
index 174856f..c7f5afa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
@@ -124,7 +124,7 @@
               Customer as customer
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
-        with  sum as tpch.coll_sum((
+        with  sum as tpch.strict_sum((
               select element o.o_totalprice
               from  orders as o
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
index cca0211..5ef7be6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
@@ -40,10 +40,10 @@
 create  dataset Orders(OrderType) primary key oid on group1;
 
 write output to asterix_nc1:"/tmp/orders-aggreg.adm";
-select element {'cid':cid,'ordpercust':`orders-aggreg`.coll_count(g),'totalcust':`orders-aggreg`.coll_sum((
+select element {'cid':cid,'ordpercust':`orders-aggreg`.strict_count(g),'totalcust':`orders-aggreg`.strict_sum((
         select element i.o.total
         from  g as i
-    )),'avgcust':`orders-aggreg`.coll_avg((
+    )),'avgcust':`orders-aggreg`.strict_avg((
         select element i.o.total
         from  g as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
index 6497eea..db8d8cd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
@@ -48,28 +48,28 @@
 load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
 
 write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.coll_sum((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':tpch.coll_sum((
+    )),'sum_base_price':tpch.strict_sum((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':tpch.coll_sum((
+    )),'sum_disc_price':tpch.strict_sum((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':tpch.coll_sum((
+    )),'sum_charge':tpch.strict_sum((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':tpch.coll_avg((
+    )),'ave_qty':tpch.strict_avg((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':tpch.coll_avg((
+    )),'ave_price':tpch.strict_avg((
         select element i.l.l_extendedprice
         from  g as i
-    )),'ave_disc':tpch.coll_avg((
+    )),'ave_disc':tpch.strict_avg((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':tpch.coll_count(g)}
+    )),'count_order':tpch.strict_count(g)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
index 9c33b03..509ed18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
@@ -86,7 +86,7 @@
 where (((c.c_mktsegment = 'BUILDING') and (c.c_custkey = o.o_custkey)) and ((l.l_orderkey = o.o_orderkey) and (o.o_orderdate < '1995-03-15') and (l.l_shipdate > '1995-03-15')))
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
-with  revenue as q3_shipping_priority.coll_sum((
+with  revenue as q3_shipping_priority.strict_sum((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  l as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
index b90b3ed..fd520e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
@@ -137,7 +137,7 @@
 where ((c.c_nationkey = o1.s_nationkey) and (c.c_custkey = o1.o_custkey))
 /* +hash */
 group by o1.n_name as n_name
-with  revenue as q5_local_supplier.coll_sum((
+with  revenue as q5_local_supplier.strict_sum((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  o1 as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
index 22bff52..d058dd3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
@@ -86,18 +86,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       with  phone_substr as tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.coll_count(g),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.strict_count(g),'totacctbal':tpch.strict_sum((
         select element i.ct.c_acctbal
         from  g as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (tpch.coll_count((
+where (tpch.strict_count((
     select element o
     from  Orders as o
     where (ct.c_custkey = o.o_custkey)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
index 2ce1638..eb73380 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
@@ -37,7 +37,7 @@
 
 create  dataset test(TestType) primary key key1;
 
-select element {'gid':aid,'avg':test.coll_avg((
+select element {'gid':aid,'avg':test.strict_avg((
         select element j.`value`
         from  i as j
         where j.`value` is not null
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
index ec19977..ee054f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
@@ -105,7 +105,7 @@
             Orders as orders
       where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = n.n_nationkey))
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
-      with  sum as tpch.coll_sum((
+      with  sum as tpch.strict_sum((
             select element o.o_totalprice
             from  orders as o
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
index a43ca13..bb54619 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
@@ -51,7 +51,7 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
index aacbb72..0accf10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
@@ -51,7 +51,7 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.coll_count(cheap),'count_expensives':tpch.coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.strict_count(cheap),'count_expensives':tpch.strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
index e1d8b97..cc7231f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
@@ -51,30 +51,30 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':tpch.coll_count((
+)),'count_order':tpch.strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
index a32530a..c77559a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
@@ -45,12 +45,12 @@
 with  lonelyusers as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   ),
       lonelyusers2 as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   )
 select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
 from  lonelyusers as l1,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
index c83a85c..59ddab8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$38]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$39]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$34]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$35]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
index 07fde95..586d5fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
@@ -2,16 +2,16 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$24]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$52(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+              -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
@@ -19,7 +19,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
index 8e1be33..d473d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
@@ -6,8 +6,8 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
+                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$26]  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index 4173afc..aeaf88f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
index 1917413..5163d81 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$68(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -17,13 +17,13 @@
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$64]  |PARTITIONED|
+                                      -- SORT_GROUP_BY[$$66]  |PARTITIONED|
                                               {
                                                 -- AGGREGATE  |LOCAL|
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
-                                        -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$57]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$66]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$59]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
index ef8a923..a050d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
index 116ff8a..180edb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$10][$$11]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$26][$$27]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
index 5cb092c..1e82aa4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
@@ -1,11 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- SORT_GROUP_BY[$$6]  |PARTITIONED|
+    -- SORT_GROUP_BY[$$27]  |PARTITIONED|
             {
               -- AGGREGATE  |LOCAL|
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
             }
-      -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+      -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
index b37868e..d3cf6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
@@ -3,22 +3,22 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$16]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$80]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$16][$$18]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$80][$$82]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
index f4a21e2..4deb445 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
@@ -12,7 +12,7 @@
                       -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$9][$$11]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$17][$$19]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
new file mode 100644
index 0000000..d38f1d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$102, $$103]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$102, $$103]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$l_returnflag, $$l_linestatus]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$92, $$93, $$94]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$92, $$93, $$94]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
index 629a05a..5ce7537 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
@@ -3,27 +3,27 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$37(DESC) ]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$38(DESC) ]  |PARTITIONED|
             -- STREAM_LIMIT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 2147483647] [$$37(DESC)]  |PARTITIONED|
+                -- STABLE_SORT [topK: 2147483647] [$$38(DESC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- STREAM_SELECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$39]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- STREAM_SELECT  |LOCAL|
                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                   }
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$39][$#1]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$40][$#1]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- STREAM_SELECT  |PARTITIONED|
@@ -31,7 +31,7 @@
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- BTREE_SEARCH  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                                        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
index d3510b3..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
@@ -1,93 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$42, $$44]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$42(ASC), $$44(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$42, $$44]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$8][$$17]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$8]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$46(ASC), $$5(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$3][$$26]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |UNPARTITIONED|
-                                                  -- ASSIGN  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$47(ASC), $$14(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$12][$$35]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
index 93fe350..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
@@ -1,57 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- IN_MEMORY_STABLE_SORT [$$4(ASC)]  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- IN_MEMORY_HASH_JOIN [$$2][$$7]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- UNNEST  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- SORT_MERGE_EXCHANGE [$$28(ASC), $$7(ASC) ]  |PARTITIONED|
-                                      -- STABLE_SORT [$$28(ASC), $$7(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EXTERNAL_GROUP_BY[$$36]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                                              -- EXTERNAL_GROUP_BY[$$6]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- UNNEST  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
index 0d7d538..254bf24 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$59]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$50]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,8 +17,8 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$49][$$52]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$50][$$53]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
@@ -26,7 +26,7 @@
                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
index e11e2a8..855fd33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -4,7 +4,7 @@
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- MATERIALIZE  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
index 08635f7..75444af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
@@ -1,11 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$36(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$14][$$15]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -13,7 +13,7 @@
                         -- DATASOURCE_SCAN  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$13(ASC)] HASH:[$$15]  |PARTITIONED|
+              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$36(ASC)] HASH:[$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
index 5aad07e..01f4b08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
index c8aaaabc..6037e0c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$7(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
index a1426f3..2785605 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$6(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
index 362a1c4..fccc3c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
index ff74295..cafb02c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
index 503fd28..96493a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
index 6194e08..19ce662 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 3e4c9bc..11cbac1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 736928c..de31517 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$20]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$43]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -22,7 +22,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                         -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 3e4c9bc..11cbac1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 5133569..74fc277 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- BTREE_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 5133569..74fc277 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- BTREE_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$20]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$43]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -22,7 +22,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                         -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 5b08bf5..5134f39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
@@ -3,18 +3,18 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$25]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$51]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+              -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$36][$$25]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$62][$$51]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
@@ -24,14 +24,14 @@
                                     -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- BTREE_SEARCH  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
index 7065d9e..97c2b45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
@@ -2,19 +2,19 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$63(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$43][$$27]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                           -- UNION_ALL  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
@@ -36,7 +36,7 @@
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
index 5668704..8f1ca57 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
@@ -2,19 +2,19 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$63(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$43][$$27]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                           -- STREAM_SELECT  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
@@ -33,7 +33,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- BTREE_SEARCH  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                               -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index 7465d81..8c0aec1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index 968246d..0d63ee1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$17]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$39]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index 7465d81..8c0aec1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index 968246d..0d63ee1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index d667093..8f3d988 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 509149f..d5e5925 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index d667093..8f3d988 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 509149f..d5e5925 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$49][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
index 0cd8363..53c00eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$37][$$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$60][$$47]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -26,7 +26,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$17]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$39]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
index 64eb060..d6e272b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
@@ -4,8 +4,8 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$17, $$22, $$24][$$19, $$23, $$20]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17, $$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$33, $$38, $$40][$$35, $$39, $$36]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$33, $$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
index 92adb3d..49e2b90 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$17, $$23, $$18][$$19, $$22, $$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$33, $$39, $$34][$$35, $$38, $$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$19, $$24]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$35, $$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
index d79e1f2..12abcc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$17, $$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$40, $$41]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
                         -- STREAM_SELECT  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$17, $$18]  |PARTITIONED|
+                -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$40, $$41]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$21, $$17, $$25][$$22, $$19, $$20]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$17, $$25]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$44, $$40, $$48][$$45, $$42, $$43]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$40, $$48]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
index c1cd283..a82ff05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
@@ -4,25 +4,25 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$17, $$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$40, $$41]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
                         -- STREAM_SELECT  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$22, $$17, $$18][$$21, $$19, $$25]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$45, $$40, $$41][$$44, $$42, $$48]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$19, $$25]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$42, $$48]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
index 08b156e..3bb5c88 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
@@ -4,10 +4,10 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$56]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
-                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$19]  |LOCAL|
+                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$57]  |LOCAL|
                                 {
                                   -- AGGREGATE  |LOCAL|
                                     -- STREAM_SELECT  |LOCAL|
@@ -17,24 +17,24 @@
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$18(ASC), $$19(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                -- STABLE_SORT [$$56(ASC), $$57(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$19][$$20]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$18][$$22]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$56][$$60]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
index b0d8e64..8d5343d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
@@ -6,15 +6,15 @@
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$18][$$19]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
index 3cc71e0..8d02861 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
@@ -3,26 +3,26 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$12]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$36]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$15][$$14]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$39][$$38]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
index 1aca471..db613d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$78][$$106]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$169][$$197]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,15 +12,15 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$197]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$103][$$85]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$194][$$176]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$79][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$170][$$172]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$170]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
@@ -28,11 +28,11 @@
                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$94][$$82]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$94]  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$185][$$173]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
@@ -40,22 +40,22 @@
                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$173]  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$91][$$83]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$182][$$174]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$182]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$89][$$84]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$180][$$175]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$180]  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
index 9f48723..d6da05e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$62][$$84]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$123][$$145]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
@@ -12,10 +12,10 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$84]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$63, $$64][$$84, $$69]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$124, $$125][$$145, $$130]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
@@ -23,15 +23,15 @@
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$84, $$69]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$145, $$130]  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$65][$$69]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$126][$$130]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$72][$$66]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -45,7 +45,7 @@
                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
index 0f8b28f..124d677 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$140, $$141]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$148, $$149]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$140, $$141]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$108, $$109]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$148, $$149]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$116, $$117]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
index 369f475..8e3ca40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$13]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$41]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$10]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$41]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$38]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
index 8908448..8b7d56e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
@@ -20,12 +20,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$14][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$38][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |UNPARTITIONED|
                   -- ASSIGN  |UNPARTITIONED|
                     -- UNNEST  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
index dd7b473..709d5d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
@@ -20,8 +20,8 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$15][$$14]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$39][$$38]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- ASSIGN  |UNPARTITIONED|
                   -- UNNEST  |UNPARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
index 3bc6933..80f1234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
@@ -2,16 +2,16 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$35(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- STREAM_SELECT  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
index 0bfec2b..1b959cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
index 807e509..0d9a257 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
index dab5a7b..e707957 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
@@ -2,8 +2,8 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$30(ASC), $$31(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$30(ASC), $$31(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$79(ASC), $$80(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$79(ASC), $$80(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- NESTED_LOOP  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
index 1ae1e60..9672972 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$146]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$162]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$123]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$139]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -21,25 +21,25 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$143]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$159]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$137]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$159]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$153]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$137(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$153(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$132][$$129]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$148][$$145]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -66,7 +66,7 @@
                                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
index 7f916f16..9dcdd24 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
@@ -2,13 +2,13 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$40(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$41(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$33]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$34]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- STREAM_SELECT  |LOCAL|
@@ -17,7 +17,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$33][$$21]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$34][$$22]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- STREAM_SELECT  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
index 7f0bb3b..2d9d64b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
@@ -2,13 +2,13 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$45(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$46(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$37]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- STREAM_SELECT  |LOCAL|
@@ -22,7 +22,7 @@
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                             }
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$37][$$36]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$38][$$37]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
@@ -30,7 +30,7 @@
                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
index 7ef3e37..170a15b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$147]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$163]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$124]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$140]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -21,25 +21,25 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$144]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$160]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$138]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$160]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$154]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$138(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$154(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$130]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$149][$$146]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -66,7 +66,7 @@
                                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
index ec9715e..9b5bedd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
@@ -5,20 +5,20 @@
         -- STREAM_PROJECT  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- STREAM_SELECT  |LOCAL|
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                       }
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$58][$#4]  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$62][$#4]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$56]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- STREAM_SELECT  |LOCAL|
@@ -27,22 +27,22 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$56][$#3]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$60][$#3]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- STREAM_SELECT  |LOCAL|
                                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$54][$#2]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$58][$#2]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+                                                          -- PRE_CLUSTERED_GROUP_BY[$$51]  |PARTITIONED|
                                                                   {
                                                                     -- AGGREGATE  |LOCAL|
                                                                       -- STREAM_SELECT  |LOCAL|
@@ -51,7 +51,7 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- HYBRID_HASH_JOIN [$$47][$#1]  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$51][$#1]  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
index a647c7c..0fabd6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
@@ -4,21 +4,21 @@
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$110(ASC) ]  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$113(ASC) ]  |PARTITIONED|
               -- STREAM_LIMIT  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [topK: 100] [$$110(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [topK: 100] [$$113(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- STREAM_SELECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- SORT_GROUP_BY[$$120]  |PARTITIONED|
+                            -- SORT_GROUP_BY[$$123]  |PARTITIONED|
                                     {
                                       -- AGGREGATE  |LOCAL|
                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                     }
-                              -- HASH_PARTITION_EXCHANGE [$$120]  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$98]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$101]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -26,33 +26,33 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$102][$$105]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$102]  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$105][$$108]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$105]  |PARTITIONED|
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- HYBRID_HASH_JOIN [$$111][$$104]  |PARTITIONED|
-                                                  -- HASH_PARTITION_EXCHANGE [$$111]  |PARTITIONED|
+                                                -- HYBRID_HASH_JOIN [$$114][$$107]  |PARTITIONED|
+                                                  -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$101][$$116]  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$101]  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$104][$$119]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$104]  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- HYBRID_HASH_JOIN [$$100][$$114]  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$103][$$117]  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$117]  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
index 906e52f..c27116d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
@@ -8,7 +8,7 @@
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$149]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- AGGREGATE  |LOCAL|
@@ -20,9 +20,9 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$144][$$145]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$149][$$150]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$132]  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$137]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- AGGREGATE  |LOCAL|
@@ -32,13 +32,13 @@
                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$132(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$137(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$132][$$133]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$137][$$138]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
+                                                -- PRE_CLUSTERED_GROUP_BY[$$117]  |PARTITIONED|
                                                         {
                                                           -- AGGREGATE  |LOCAL|
                                                             -- AGGREGATE  |LOCAL|
@@ -48,7 +48,7 @@
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$112][$$122]  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$117][$$127]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -66,7 +66,7 @@
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- REPLICATE  |PARTITIONED|
-                                                                    -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
@@ -78,18 +78,18 @@
                                                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$138]  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$126][$$125]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$131][$$130]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- STREAM_SELECT  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$134]  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$139]  |PARTITIONED|
                                                                             {
                                                                               -- AGGREGATE  |LOCAL|
                                                                                 -- AGGREGATE  |LOCAL|
@@ -99,7 +99,7 @@
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$134][$$136]  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$139][$$141]  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                   -- ASSIGN  |PARTITIONED|
@@ -115,7 +115,7 @@
                                                                                   -- ASSIGN  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                       -- REPLICATE  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                                        -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
@@ -127,7 +127,7 @@
                                                                                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$125]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
@@ -140,13 +140,13 @@
                                                                             -- DATASOURCE_SCAN  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$128][$$127]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$146]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$133][$$132]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$151]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- AGGREGATE  |LOCAL|
@@ -155,18 +155,18 @@
                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$146(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$151(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$146][$$149]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$151][$$154]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- STREAM_SELECT  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ASSIGN  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$150]  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
                                                                             {
                                                                               -- AGGREGATE  |LOCAL|
                                                                                 -- AGGREGATE  |LOCAL|
@@ -176,7 +176,7 @@
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$150][$$151]  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$155][$$156]  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -194,7 +194,7 @@
                                                                                   -- ASSIGN  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                       -- REPLICATE  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                                        -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
@@ -206,18 +206,18 @@
                                                                                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$154]  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- HYBRID_HASH_JOIN [$$163][$$162]  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$168][$$167]  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$168]  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- STREAM_SELECT  |PARTITIONED|
                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- PRE_CLUSTERED_GROUP_BY[$$168]  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$173]  |PARTITIONED|
                                                                                       {
                                                                                         -- AGGREGATE  |LOCAL|
                                                                                           -- AGGREGATE  |LOCAL|
@@ -227,7 +227,7 @@
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- HYBRID_HASH_JOIN [$$168][$$169]  |PARTITIONED|
+                                                                                      -- HYBRID_HASH_JOIN [$$173][$$174]  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                           -- REPLICATE  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -237,7 +237,7 @@
                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                           -- REPLICATE  |PARTITIONED|
-                                                                                            -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                                   -- ASSIGN  |PARTITIONED|
@@ -249,7 +249,7 @@
                                                                                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$167]  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -260,7 +260,7 @@
                                                                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
index c335414..67965bf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
@@ -16,7 +16,7 @@
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$141]  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$146]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- AGGREGATE  |LOCAL|
@@ -28,9 +28,9 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$141][$$142]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$146][$$147]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$82]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- AGGREGATE  |LOCAL|
@@ -40,12 +40,12 @@
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$77][$$134]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$82][$$139]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
                                                   -- STREAM_PROJECT  |UNPARTITIONED|
                                                     -- ASSIGN  |UNPARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -63,7 +63,7 @@
                                                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$134]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- STREAM_SELECT  |PARTITIONED|
@@ -94,7 +94,7 @@
                                                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
@@ -108,7 +108,7 @@
                                                       -- STREAM_SELECT  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$143]  |PARTITIONED|
+                                                            -- PRE_CLUSTERED_GROUP_BY[$$148]  |PARTITIONED|
                                                                     {
                                                                       -- AGGREGATE  |LOCAL|
                                                                         -- AGGREGATE  |LOCAL|
@@ -117,12 +117,12 @@
                                                                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                     }
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$143(ASC)]  |PARTITIONED|
+                                                                -- STABLE_SORT [$$148(ASC)]  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$143][$$144]  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$148][$$149]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |UNPARTITIONED|
                                                                               -- ASSIGN  |UNPARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -140,7 +140,7 @@
                                                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                 -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
index c2790a4..2033230 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$119]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$121]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$101]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$103]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,12 +17,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$101][$$108]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$103][$$110]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$110][$$105]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$112][$$107]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$112]  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
index 789992e..61d53a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$119]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$121]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$101]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$103]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,11 +17,11 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$101][$$108]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$103][$$110]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$109][$$105]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$111][$$107]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
index e9fafd7..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
@@ -1,33 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$20(ASC), $$18(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$3][$$22]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
index f65ede3..da7ba31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
@@ -35,5 +35,10 @@
         <output-dir compare="Text">secondary-index</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="metrics">
+      <compilation-unit name="secondary-index-index-only">
+        <output-dir compare="Text">secondary-index-index-only</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
deleted file mode 100644
index bcc0edc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-//polltimeoutsecs=10
-
-/query/service/status/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
new file mode 100644
index 0000000..286c097
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+//polltimeoutsecs=10
+// handlevariable=handle
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
new file mode 100644
index 0000000..b59be3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c2 div $c1,"result2": $c2 div $c2,"result3": $c2 div $c3,"result4": $c2 div $c4,"result5": $c2 div $c5, "result6": $c2 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
new file mode 100644
index 0000000..d8ad04a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c3 div $c1,"result2": $c3 div $c2,"result3": $c3 div $c3,"result4": $c3 div $c4,"result5": $c3 div $c5, "result6": $c3 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
new file mode 100644
index 0000000..32c72ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c4 div $c1,"result2": $c4 div $c2,"result3": $c4 div $c3,"result4": $c4 div $c4,"result5": $c4 div $c5, "result6": $c4 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
new file mode 100644
index 0000000..5adb1b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c1 div $c1,"result2": $c1 div $c2,"result3": $c1 div $c3,"result4": $c1 div $c4,"result5": $c1 div $c5, "result6": $c1 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index bc9e7f7..0ab61e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -36,11 +36,11 @@
 let $str8:=substring($str7,0)
 
 let $str9:="This is a test string"
-let $str10:=substring($str9,-1)
+let $str10:=substring($str9,-6)
 
 let $str11:="This is a test string"
 let $str12:="This is a another test string"
 let $str13:=substring(string-concat([$str11,$str12]),20)
 
-let $str14:=substring("UC Irvine",string-length("UC Irvine")/2 - 1)
+let $str14:=substring("UC Irvine",string-length("UC Irvine") div 2 - 1)
 return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
index 075af82..ecfcd94 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
@@ -18,5 +18,5 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",-1)
+let $c1 := substring("HEllow",-3)
 return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
index ed067b2..ca84a8b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
index 9b0eb7d..3faf34f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
index 53f0cdb..fe6d3cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
index 9b412c3..98b06a8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
index 00f93a4..7cdf71b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
index 9424486..c8e7cf8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
index 38c0972..947d19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
@@ -22,4 +22,4 @@
 * Date         : Jun 2nd 2013
 */
 
-{'count1':coll_count([null]),'average1':coll_avg([null]),'sum1':coll_sum([null]),'min1':coll_min([null]),'max1':coll_max([null]),'count2':coll_count({{null,null}}),'average2':coll_avg({{null,null}}),'sum2':coll_sum({{null,null}}),'min2':coll_min({{null,null}}),'max2':coll_max({{null,null}})};
+{'count1':strict_count([null]),'average1':strict_avg([null]),'sum1':strict_sum([null]),'min1':strict_min([null]),'max1':strict_max([null]),'count2':strict_count({{null,null}}),'average2':strict_avg({{null,null}}),'sum2':strict_sum({{null,null}}),'min2':strict_min({{null,null}}),'max2':strict_max({{null,null}})};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
index 9ada73b..62506cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
@@ -25,19 +25,19 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t.valplus
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element t.valplus
     from  Test as t
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element t.valplus
     from  Test as t
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element t.valplus
     from  Test as t
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element t.valplus
     from  Test as t
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
index 3643901..47887d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
@@ -25,28 +25,28 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element i.val
     from  (
         select element t
         from  Test as t
     ) as i
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element i.val
     from  (
         select element t
         from  Test as t
     ) as i
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element i.valplus
     from  (
         select element t
         from  Test as t
     ) as i
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element i.valplus
     from  (
         select element t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
index c90641b..5577347 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
@@ -22,4 +22,4 @@
 * Date         : Jun 2nd 2013
 */
 
-{'count1':coll_count([float('2.0'),double('3.0'),93847382783847382,1]),'average1':coll_avg([float('2.0'),double('3.0'),93847382783847382,1]),'sum1':coll_sum([float('2.0'),double('3.0'),93847382783847382,1]),'min1':coll_min([float('2.0'),double('3.0'),93847382783847382,1]),'max1':coll_max([float('2.0'),double('3.0'),93847382783847382,1]),'count2':coll_count({{float('2.0'),double('3.0'),93847382783847382,1}}),'average2':coll_avg({{float('2.0'),double('3.0'),93847382783847382,1}}),'sum2':coll_sum({{float('2.0'),double('3.0'),93847382783847382,1}}),'min2':coll_min({{float('2.0'),double('3.0'),93847382783847382,1}}),'max2':coll_max({{float('2.0'),double('3.0'),93847382783847382,1}})};
+{'count1':strict_count([float('2.0'),double('3.0'),93847382783847382,1]),'average1':strict_avg([float('2.0'),double('3.0'),93847382783847382,1]),'sum1':strict_sum([float('2.0'),double('3.0'),93847382783847382,1]),'min1':strict_min([float('2.0'),double('3.0'),93847382783847382,1]),'max1':strict_max([float('2.0'),double('3.0'),93847382783847382,1]),'count2':strict_count({{float('2.0'),double('3.0'),93847382783847382,1}}),'average2':strict_avg({{float('2.0'),double('3.0'),93847382783847382,1}}),'sum2':strict_sum({{float('2.0'),double('3.0'),93847382783847382,1}}),'min2':strict_min({{float('2.0'),double('3.0'),93847382783847382,1}}),'max2':strict_max({{float('2.0'),double('3.0'),93847382783847382,1}})};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
index 42306d8..c7d7eb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
@@ -25,19 +25,19 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t.valplus
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element t.valplus
     from  Test as t
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element t.valplus
     from  Test as t
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element t.valplus
     from  Test as t
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element t.valplus
     from  Test as t
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
index c9a79fb..816f452 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': to_bigint(coll_avg(distinct [10,20,30,10,20,30,10])),
-   't2': to_bigint(coll_avg(distinct [null,missing,null,missing])),
-   't3': to_bigint(coll_avg(distinct [40,null,missing,50,40,null,missing,50,40])),
-   't4': to_bigint(coll_avg(distinct []))
- }
\ No newline at end of file
+   't1': to_bigint(strict_avg(distinct [10,20,30,10,20,30,10])),
+   't2': to_bigint(strict_avg(distinct [null,missing,null,missing])),
+   't3': to_bigint(strict_avg(distinct [40,null,missing,50,40,null,missing,50,40])),
+   't4': to_bigint(strict_avg(distinct []))
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
index ae1b1eb..a10da57 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-select element coll_avg((
+select element strict_avg((
     select element x
     from  [1.0,2.0,double('3.0')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
index 4c10066..a6a4537 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.doubleField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
index d7a5263..05d6092 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
index 8d2359f..7eb6e35 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
index 14c4bc5..c13d9a1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.float('1'),test.float('2'),test.float('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
index c8c3774..a694993 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.floatField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
index 6883fa8..90f54cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.smallint('1'),test.smallint('2'),test.smallint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
index d50b0d1..4e2e8c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int16Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
index a235c84..d52ebde 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [1,2,3] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
index 9618d2d..34c5c68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int32Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
index 813c9a668e..5afa4e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.bigint('1'),test.bigint('2'),test.bigint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
index 8c063d3..230913f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int64Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
index 3d2c5ba..57ffb31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
index 74fb4c4..64b91ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int8Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
index 4043084..0085e3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_avg((
+select element strict_avg((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
index fa90f85..2bf8960 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element x
     from  [1,2,3] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
index a7d9b90..403f8b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_count(distinct [10,20,30,10,20,30,10]),
-   't2': coll_count(distinct [null,missing,null,missing]),
-   't3': coll_count(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_count(distinct [])
- }
\ No newline at end of file
+   't1': strict_count(distinct [10,20,30,10,20,30,10]),
+   't2': strict_count(distinct [null,missing,null,missing]),
+   't3': strict_count(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_count(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
index 383671d..312bfe0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Success        : Yes
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
index 67e6e77..a324793 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
index ec84f7c..406a0c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element x.doubleField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
index feea992..cb308cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element l.name
     from  Employee as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
index deaa68a..1af45c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_count(['ASTERIX','Hyracks',null]);
+strict_count(['ASTERIX','Hyracks',null]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
index 9da7811..b1e3de5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-{'count':coll_count([1,60,null]),'average':coll_avg([1,60,null]),'sum':coll_sum([1,60,null]),'min':coll_min([1,60,null]),'max':coll_max([1,60,null])};
+{'count':strict_count([1,60,null]),'average':strict_avg([1,60,null]),'sum':strict_sum([1,60,null]),'min':strict_min([1,60,null]),'max':strict_max([1,60,null])};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
index b2c6c66..9ecf906 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_min([23,748374857483]);
+strict_min([23,748374857483]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
index e12e644..a0134a6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_min([748374857483,23,0.5]);
+strict_min([748374857483,23,0.5]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
index 770b7dc..0f82177 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_sum([23,748374857483]);
+strict_sum([23,748374857483]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
index 3932e23..f6a1989 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_sum([748374857483,23,0.5]);
+strict_sum([748374857483,23,0.5]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
index 2a3312e..25cef91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
@@ -27,10 +27,10 @@
 use test;
 
 
-select element {'min':test.coll_min((
+select element {'min':test.strict_min((
         select element l.name
         from  t1 as l
-    )),'max':test.coll_max((
+    )),'max':test.strict_max((
         select element l.name
         from  t1 as l
     ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
index 70f273f..8ceef73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_max(distinct [10,20,30,10,20,30,10]),
-   't2': coll_max(distinct [null,missing,null,missing]),
-   't3': coll_max(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_max(distinct [])
- }
\ No newline at end of file
+   't1': strict_max(distinct [10,20,30,10,20,30,10]),
+   't2': strict_max(distinct [null,missing,null,missing]),
+   't3': strict_max(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_max(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
index fa419e6..6e3e097 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_max((
+select element test.strict_max((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
index 1ad7c2b..36c14f9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_max((
+select element test.strict_max((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
index 77684a0..e02eac6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_min(distinct [10,20,30,10,20,30,10]),
-   't2': coll_min(distinct [null,missing,null,missing]),
-   't3': coll_min(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_min(distinct [])
- }
\ No newline at end of file
+   't1': strict_min(distinct [10,20,30,10,20,30,10]),
+   't2': strict_min(distinct [null,missing,null,missing]),
+   't3': strict_min(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_min(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
index aa84453..2840b0f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_min((
+select element test.strict_min((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
index bd686b6..bc4fe9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_min((
+select element test.strict_min((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
index 1802fee..c55fb2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_min((
+select element strict_min((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
index f910f12..03edd51 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
@@ -23,6 +23,6 @@
  * Date         : 8th May 2013
  */
 
-coll_count((select element i
+strict_count((select element i
 from  [[1,2,3,4,5],[6,7,8,9]] as i
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
index b62910e..901517d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_avg([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_avg([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_avg([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_avg([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_avg([test.double('1'),test.double('2'),test.double('3')])
+with  i8 as test.strict_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_avg([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_avg([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_avg([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_avg([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_avg([test.double('1'),test.double('2'),test.double('3')])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
index d2655e4..60f4d9d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
@@ -21,4 +21,4 @@
  * Success        : Yes
  */
 
-select element coll_avg([]);
+select element strict_avg([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
index 254deb0..4e08eba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_avg([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_avg([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_avg([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_avg([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_avg([test.double('1'),test.double('2'),test.double('3'),null])
+with  i8 as test.strict_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_avg([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_avg([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_avg([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_avg([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_avg([test.double('1'),test.double('2'),test.double('3'),null])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
index a1a34a1..91650c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
@@ -24,13 +24,13 @@
 use test;
 
 
-with  i8 as test.coll_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_count([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_count([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_count([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_count([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_count([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_count(['a','b','c'])
+with  i8 as test.strict_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_count([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_count([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_count([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_count([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_count([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_count(['a','b','c'])
 select element i
 from  [i8,i16,i32,i64,f,d,s] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
index d2fff80..d7e966d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_count([]);
+select element test.strict_count([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
index 3892816..2d29ce0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
@@ -24,13 +24,13 @@
 use test;
 
 
-with  i8 as test.coll_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_count([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_count([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_count([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_count([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_count([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_count(['a','b','c',null])
+with  i8 as test.strict_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_count([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_count([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_count([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_count([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_count([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_count(['a','b','c',null])
 select element i
 from  [i8,i16,i32,i64,f,d,s] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
index ebcd83b..3892f59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_max([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_max([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_max([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_max([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_max([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_max(['foo','bar','world']),
-      dt as test.coll_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
+with  i8 as test.strict_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_max([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_max([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_max([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_max([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_max([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_max(['foo','bar','world']),
+      dt as test.strict_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
index 7978919..98175ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_max([]);
+select element test.strict_max([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
index b5bd86e..e4d9123 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_max([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_max([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_max([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_max([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_max([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_max(['foo','bar','world',null]),
-      dt as test.coll_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
+with  i8 as test.strict_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_max([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_max([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_max([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_max([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_max([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_max(['foo','bar','world',null]),
+      dt as test.strict_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
index d2e07b6..bb7d0db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_min([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_min([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_min([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_min([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_min([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_min(['foo','bar','world']),
-      dt as test.coll_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
+with  i8 as test.strict_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_min([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_min([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_min([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_min([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_min([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_min(['foo','bar','world']),
+      dt as test.strict_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
index 8a9f7ec..df4b788 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_min([]);
+select element test.strict_min([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
index 97cca36..3460ba8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_min([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_min([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_min([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_min([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_min([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_min(['foo','bar','world',null]),
-      dt as test.coll_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
+with  i8 as test.strict_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_min([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_min([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_min([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_min([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_min([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_min(['foo','bar','world',null]),
+      dt as test.strict_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
index 300860f..a35d8a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_sum([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_sum([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_sum([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_sum([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_sum([test.double('1'),test.double('2'),test.double('3')])
+with  i8 as test.strict_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_sum([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_sum([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_sum([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_sum([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_sum([test.double('1'),test.double('2'),test.double('3')])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
index a0d5516..4de084e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_sum([]);
+select element test.strict_sum([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
index 70ba6ad..70778e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_sum([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_sum([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_sum([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_sum([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_sum([test.double('1'),test.double('2'),test.double('3'),null])
+with  i8 as test.strict_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_sum([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_sum([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_sum([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_sum([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_sum([test.double('1'),test.double('2'),test.double('3'),null])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
index 6991a74..01f6cef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_sum(distinct [10,20,30,10,20,30,10]),
-   't2': coll_sum(distinct [null,missing,null,missing]),
-   't3': coll_sum(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_sum(distinct [])
- }
\ No newline at end of file
+   't1': strict_sum(distinct [10,20,30,10,20,30,10]),
+   't2': strict_sum(distinct [null,missing,null,missing]),
+   't3': strict_sum(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_sum(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
index e8a1f4d..cff9f78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [1.0,2.0,3.0] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
index 7a17bab..7268e26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.doubleField
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
index bfc0e2c..e269827 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
index ce6abf4..bc029a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
index 8f90964..8931bb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.float('1'),test.float('2'),test.float('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
index f4c25bb..e7e2d39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.floatField
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
index 5147881..57bf229 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.smallint('1'),test.smallint('2'),test.smallint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
index e57c3f4..a9e5336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int16Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
index 616e7d1..43ae045 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.integer('1'),test.integer('2'),test.integer('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
index 3c6841e..8732973 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int32Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
index 503b0a8..243428a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.bigint('1'),test.bigint('2'),test.bigint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
index 857e15f..f19157b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int64Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
index b22d31b..595ecca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
index 473bb1a..ab30e0d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int8Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
index b8c98ef..175fcee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_sum((
+select element strict_sum((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
index 0cb9c60..0354a48 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
index fdc8b57..d3ec516 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
index 263ce45..449c361 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
@@ -26,7 +26,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element l.sal
     from  tdst as l
     where l.sal IS NOT NULL
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
index 27624da..1c98590 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
index 10f6c7e..c30742c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
index 7b771d7..2d3f2e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element l.sal
     from  tdst as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
index 1ae94ec..e5a1c1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
@@ -53,6 +53,7 @@
         <compilation-unit name="async-exhausted-result">
             <output-dir compare="Text">async-exhausted-result</output-dir>
             <expected-error>HYR0093</expected-error>
+            <source-location>false</source-location>
         </compilation-unit>
     </test-case>
 </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp
new file mode 100644
index 0000000..c11dd76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": missingif(missing, missing) is missing,
+  "t2": missingif(missing, 2) is missing,
+  "t3": missingif(2, missing) is missing,
+  "t4": missingif(missing, null) is missing,
+  "t5": missingif(null, missing) is missing,
+  "t6": missingif(null, null) is null,
+  "t7": missingif(null, 2) is null,
+  "t8": missingif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp
new file mode 100644
index 0000000..b5f08ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_missing(missing_if(t1[1], t2[1]), "MISSING") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp
new file mode 100644
index 0000000..44e64d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": nanif(missing, missing) is missing,
+  "t2": nanif(missing, 2) is missing,
+  "t3": nanif(2, missing) is missing,
+  "t4": nanif(missing, null) is missing,
+  "t5": nanif(null, missing) is missing,
+  "t6": nanif(null, null) is null,
+  "t7": nanif(null, 2) is null,
+  "t8": nanif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp
new file mode 100644
index 0000000..05e003e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(nan_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp
new file mode 100644
index 0000000..abd4a4b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": neginfif(missing, missing) is missing,
+  "t2": neginfif(missing, 2) is missing,
+  "t3": neginfif(2, missing) is missing,
+  "t4": neginfif(missing, null) is missing,
+  "t5": neginfif(null, missing) is missing,
+  "t6": neginfif(null, null) is null,
+  "t7": neginfif(null, 2) is null,
+  "t8": neginfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp
new file mode 100644
index 0000000..67dcd34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(neginf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp
new file mode 100644
index 0000000..69ef8e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": nullif(missing, missing) is missing,
+  "t2": nullif(missing, 2) is missing,
+  "t3": nullif(2, missing) is missing,
+  "t4": nullif(missing, null) is missing,
+  "t5": nullif(null, missing) is missing,
+  "t6": nullif(null, null) is null,
+  "t7": nullif(null, 2) is null,
+  "t8": nullif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp
new file mode 100644
index 0000000..d5d6be3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_null(null_if(t1[1], t2[1]), "NULL") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp
new file mode 100644
index 0000000..399f4d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": posinfif(missing, missing) is missing,
+  "t2": posinfif(missing, 2) is missing,
+  "t3": posinfif(2, missing) is missing,
+  "t4": posinfif(missing, null) is missing,
+  "t5": posinfif(null, missing) is missing,
+  "t6": posinfif(null, null) is null,
+  "t7": posinfif(null, 2) is null,
+  "t8": posinfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp
new file mode 100644
index 0000000..cf25c1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(posinf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp
new file mode 100644
index 0000000..9083866
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.SupplierType as closed {
+  s_suppkey : int32,
+  s_name : string,
+  s_address : string,
+  s_nationkey : int32,
+  s_phone : string,
+  s_acctbal : float,
+  s_comment : string
+};
+
+create  dataset Supplier(SupplierType) primary key s_name;
+
+
+load  dataset Supplier using localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp
new file mode 100644
index 0000000..128f64f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+select value s from Supplier s where 1 = s.s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp
new file mode 100644
index 0000000..f7ed2d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+select value s from Supplier s where s.s_suppkey <= 5 order by s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp
new file mode 100644
index 0000000..8d10856
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+select value s from Supplier s where s.s_nationkey = 11;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp
new file mode 100644
index 0000000..6412265
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use tpch;
+
+select value s from Supplier s where s.s_acctbal < 4500 order by s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp
new file mode 100644
index 0000000..21d5a43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse tpch;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
index 51668ed..7c6bfa8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
@@ -26,7 +26,7 @@
  * Date         : 28th Aug 2012
  */
 
-select element coll_count((
+select element strict_count((
     select element l
     from  `Metadata`.`Dataset` as l
     where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
index 413ab84..bc0e6c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
@@ -24,7 +24,7 @@
 
 SELECT c.name AS cust_name,
        c.cashBack AS cust_cashBack
-FROM customer c JOIN [coll_min((SELECT ELEMENT c.cashBack FROM customer c))] as min_cashBack
+FROM customer c JOIN [strict_min((SELECT ELEMENT c.cashBack FROM customer c))] as min_cashBack
      ON c.cashBack = min_cashBack
 ORDER BY c.cid, c.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
index 5f4f07f..7e89b59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
@@ -29,7 +29,7 @@
 
 SELECT c.name AS cust_name,
        c.cashBack AS cust_cashBack
-FROM customer c JOIN [coll_min((SELECT ELEMENT c.cashBack FROM c c))] as min_cashBack
+FROM customer c JOIN [strict_min((SELECT ELEMENT c.cashBack FROM c c))] as min_cashBack
      ON c.cashBack = min_cashBack
 ORDER BY c.cid, c.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp
new file mode 100644
index 0000000..a41976b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type sub as
+{ subscriptionId: uuid }
+;
+
+create dataset subscriptions(sub) primary key subscriptionId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp
new file mode 100644
index 0000000..dad3a82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+upsert into subscriptions as record(
+[{"subscriptionId":create_uuid(), "word": "hello"}]
+) returning record.word;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
index c49b436..dc4c880 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
@@ -20,7 +20,7 @@
 use SocialNetworkData;
 
 
-select element SocialNetworkData.coll_count((
+select element SocialNetworkData.strict_count((
     select element h
     from  HandbookUsers as h
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
index 49fb04c..cf3e9b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds;
 
 case
- when ((feeds.coll_count((select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds.strict_count((select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
- end;
\ No newline at end of file
+ end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
index f7ae3c8..cf3e9b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds;
 
 case
- when ((feeds.coll_count((select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds.strict_count((select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
  end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
index d0f5898..5b0acba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds_09;
 
 case
- when ((feeds_09.coll_count(( select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds_09.strict_count(( select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
- end;
\ No newline at end of file
+ end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
index b763e83..d55804c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
@@ -19,7 +19,7 @@
 
 use feeds_10;
 
-select element feeds_10.coll_count((
+select element feeds_10.strict_count((
     select element x
     from  Tweets as x
     order by x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
index a9ed09d..0c56f0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
@@ -27,6 +27,4 @@
 
 use experiments;
 
-select value count(*) from DBLP0;
-select value count(*) from DBLP1;
-select value count(*) from DBLP2;
\ No newline at end of file
+select value count(*) from DBLP0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
new file mode 100644
index 0000000..62b920b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date         : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
new file mode 100644
index 0000000..b7f38b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date         : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp
new file mode 100644
index 0000000..95c1c38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test filters with autogenerated keys
+ * Expected Res : Success
+ * Date         : 20th Mar 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type UserLocation as closed {
+recordId: uuid,
+location: circle,
+userName: string
+};
+
+
+create dataset UserLocations(UserLocation)
+primary key recordId autogenerated with filter on userName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp
new file mode 100644
index 0000000..cee10ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into UserLocations(
+{"userName" : "c1121u1" , "location" : circle("4171.58,1083.41 100.0")}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp
new file mode 100644
index 0000000..945fc1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select l.userName from UserLocations l
+where l.userName = "c1121u1";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
index 5b44875..d3e48c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date             :  31st July 2012
  */
 
-select element {'store-number':strNum,'total-qty':coll_sum((
+select element {'store-number':strNum,'total-qty':strict_sum((
         select element l.qty
         from  sales as l
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
index 5b44875..d3e48c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date             :  31st July 2012
  */
 
-select element {'store-number':strNum,'total-qty':coll_sum((
+select element {'store-number':strNum,'total-qty':strict_sum((
         select element l.qty
         from  sales as l
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
index aab13a4..ab85a07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
@@ -53,9 +53,9 @@
 SELECT VALUE
 {
    "user": a.user,
-   "beth": coll_count(beth),
-   "ts": coll_avg( ( FROM ts AS l SELECT VALUE l.timespent ) ),
-   "rev": coll_sum( ( FROM rev AS k SELECT VALUE k.estimated_revenue ) )
+   "beth": strict_count(beth),
+   "ts": strict_avg( ( FROM ts AS l SELECT VALUE l.timespent ) ),
+   "rev": strict_sum( ( FROM rev AS k SELECT VALUE k.estimated_revenue ) )
 }
 ORDER BY a.user;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
new file mode 100644
index 0000000..a1173de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop dataverse Coffee if exists;
+create dataverse Coffee;
+use Coffee;
+
+
+create type CoffeeType as {
+    id: string,
+    location: point
+};
+
+create type FollowersType as {
+    user_id: string,
+    twitter_id: string,
+    followers: [string]
+};
+
+create dataset Coffee(CoffeeType)
+    primary key id;
+
+create dataset Followers(FollowersType)
+    primary key user_id;
+
+create type Tweet as open {
+    id: int64,
+    timestamp_ms: string
+};
+
+create dataset Tweets (Tweet)
+primary key id;
+
+create function CoffeeWithFriends(user_id){
+(select * from Tweets t
+let temp = (select * from Tweets f where f.user.id_str in (select value fw.followers from Followers fw where fw.twitter_id=user_id)[0]
+and
+spatial_intersect(create_point(f.place.bounding_box.coordinates[0][0][0],f.place.bounding_box.coordinates[0][0][1]),
+create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1])))
+where t.user.id_str = user_id)
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp
new file mode 100644
index 0000000..7bb9131
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use Coffee;
+upsert into Coffee
+([
+{"id":"001", "location":point("-117.523867, 33.85216")},
+{"id":"002", "location":point("-122.37382985, 47.67917392")},
+{"id":"003", "location":point("-122.00795969, 37.40427696")},
+{"id":"004", "location":point("-122.00794969, 37.40427596")}
+]);
+
+upsert into Followers
+([
+{"user_id":"userId@Coffee", "twitter_id": "988455822727393280", "followers":["988455822727393281","988455822727393282", "0004", "0005", "0006", "0007", "0008"]}
+]);
+
+upsert into Tweets([
+{ "id": 999403843673718784, "user":
+{ "id": 988455822727393280, "id_str": "988455822727393280", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+ { "id": 999403845, "user":
+{ "id": 988455822727393281, "id_str": "988455822727393281", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+  { "id": 999403846, "user":
+{ "id": 988455822727393282, "id_str": "988455822727393282", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp
new file mode 100644
index 0000000..140f6c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use Coffee;
+CoffeeWithFriends("988455822727393280");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp
new file mode 100644
index 0000000..639d98d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop dataverse Coffee;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
new file mode 100644
index 0000000..24f4ed6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
@@ -0,0 +1,35 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-group name="geojson">
+    <test-case FilePath="geojson">
+        <compilation-unit name="datatype">
+            <output-dir compare="Text">datatype</output-dir>
+        </compilation-unit>
+    </test-case>
+    <test-case FilePath="geojson">
+        <compilation-unit name="single-method">
+            <output-dir compare="Text">single-method</output-dir>
+        </compilation-unit>
+    </test-case>
+    <test-case FilePath="geojson">
+        <compilation-unit name="two-geometries">
+            <output-dir compare="Text">two-geometries</output-dir>
+        </compilation-unit>
+    </test-case>
+</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
new file mode 100644
index 0000000..61a7be6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"PointN": st_point_n(geo.myGeometry,1), "StartPoint":st_start_point(geo.myGeometry),"Envelope":st_envelope(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
new file mode 100644
index 0000000..dd378a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse GeoJSON if exists;
+create  dataverse GeoJSON;
+
+use GeoJSON;
+
+CREATE TYPE GeometryType AS{
+  id : int,
+  myGeometry : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp
new file mode 100644
index 0000000..3ab3bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry": st_geom_from_geojson({"type":"Point","coordinates":[-118.4,33.93]})},
+{"id": 124, "myGeometry": st_geom_from_geojson({"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]})},
+{"id": 126, "myGeometry": st_geom_from_geojson({"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]})},
+{"id": 127, "myGeometry": st_geom_from_geojson({"type": "MultiPoint","coordinates": [[10, 40], [40, 30], [20, 20], [30, 10]]})},
+{"id": 128, "myGeometry": st_geom_from_geojson({"type": "MultiLineString","coordinates": [[[10, 10], [20, 20], [10, 40]],[[40, 40], [30, 30], [40, 20], [30, 10]]]})},
+{"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
+{"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
+{"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
+{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+{"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
+{"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
+{"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
+{"id": 136, "myGeometry": st_make_envelope(10, 10, 11, 11, 4326)},
+{"id": 137, "myGeometry": st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))")}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
new file mode 100644
index 0000000..540bce1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT * FROM Geometries ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
new file mode 100644
index 0000000..beb8879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"IsClosed":st_is_closed(geo.myGeometry), "IsCollection":st_is_collection(geo.myGeometry),"IsEmpty":st_is_empty(geo.myGeometry), "IsSimple":st_is_simple(geo.myGeometry)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp
new file mode 100644
index 0000000..4a19b47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"Type": geometry_type(geo.myGeometry), "Area": st_area(geo.myGeometry),
+"Coordinate dimension": st_coord_dim(geo.myGeometry), "Dimension":st_dimension(geo.myGeometry),
+"NPoints":st_n_points(geo.myGeometry), "XMax":st_x_max(geo.myGeometry),"XMin":st_x_min(geo.myGeometry),
+"YMax":st_y_max(geo.myGeometry), "YMin":st_y_min(geo.myGeometry), "Binary": st_as_binary(geo.myGeometry),
+"GeoJSON":st_as_geojson(geo.myGeometry),"WKT":st_as_text(geo.myGeometry)}  FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
new file mode 100644
index 0000000..4868d4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"X":st_x(geo.myGeometry),"Y":st_y(geo.myGeometry),"Z":st_z(geo.myGeometry), "M":st_m(geo.myGeometry)}  FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Point" ;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
new file mode 100644
index 0000000..d7e32a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"NumInteriorRings":st_num_interior_rings(geo.myGeometry), "ExteriorRing":st_exterior_ring(geo.myGeometry)}  FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Polygon";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
new file mode 100644
index 0000000..9fb187d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON;
+
+SELECT VALUE {"Length": st_length(geo.myGeometry), "Boundary":st_boundary(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString" OR geometry_type(geo.myGeometry)="MultiLineString" ORDER BY geo.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
new file mode 100644
index 0000000..01c2563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+st_geometry_n(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'),2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp
new file mode 100644
index 0000000..28ee45f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+st_is_collection(st_polygonize([st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")]));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp
new file mode 100644
index 0000000..54b4450
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+st_is_empty(st_geom_from_text("LINESTRING EMPTY"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
new file mode 100644
index 0000000..9924d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+st_num_geometries(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp
new file mode 100644
index 0000000..3a889b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Union": st_union(geo.myGeometry1, geo.myGeometry2),"Intersection":st_intersection(geo.myGeometry1,
+geo.myGeometry2),"SymDifference":st_sym_difference(geo.myGeometry1,geo.myGeometry2)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp
new file mode 100644
index 0000000..7084fe3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON1;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 124, "myGeometry1": st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), "myGeometry2": st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")},
+{"id": 126, "myGeometry1": st_geom_from_text('POINT(0 0)'), "myGeometry2": st_geom_from_text('LINESTRING ( 0 0, 0 2 )')},
+{"id": 127, "myGeometry1": st_geom_from_text('LINESTRING(1 1,-1 -1,2 3.5,1 3,1 2,2 1)'), "myGeometry2": st_make_point(-1, -1)},
+{"id": 128, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('LINESTRING(0 2,1 2,2 2,3 2,4 2,5 2)')},
+{"id": 129, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('POINT(0 0)')},
+{"id": 130, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 10 10)'), "myGeometry2": st_geom_from_text('LINESTRING(0 0, 5 5, 10 10)')},
+{"id": 131, "myGeometry1": st_geom_from_text('LINESTRING(1 2, 3 4)'), "myGeometry2": st_geom_from_text('LINESTRING(5 6, 7 8)')},
+{"id": 132, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 133, "myGeometry1": st_geom_from_text('POLYGON((0.25 0.25, 0.5 0.25, 0.5 0.5, 0.25 0.5, 0.25 0.25 ))'), "myGeometry2": st_geom_from_text('POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))')}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp
new file mode 100644
index 0000000..671d151
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse GeoJSON1 if exists;
+create  dataverse GeoJSON1;
+
+use GeoJSON1;
+
+CREATE TYPE GeometryType AS{
+  id : int,
+  myGeometry1 : geometry,
+  myGeometry2 : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp
new file mode 100644
index 0000000..74a3fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Distance":st_distance(geo.myGeometry1, geo.myGeometry2), "Intersects":st_intersects(geo.myGeometry1,
+geo.myGeometry2), "Contains":st_contains(geo.myGeometry1, geo.myGeometry2), "Crosses":st_crosses(geo.myGeometry1,
+geo.myGeometry2), "Disjoint":st_disjoint(geo.myGeometry1, geo.myGeometry2), "Equals":st_equals(geo.myGeometry1,
+geo.myGeometry2), "Overlaps":st_overlaps(geo.myGeometry1, geo.myGeometry2), "Relate":st_relate(geo.myGeometry1,
+geo.myGeometry2, "FF1FF0102"), "Touches":st_touches(geo.myGeometry1, geo.myGeometry2), "Within":st_within(
+geo.myGeometry1, geo.myGeometry2)} FROM Geometries geo;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
index 907afb0..5e98b19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
@@ -19,5 +19,5 @@
 
 USE TinySocial;
 
-SELECT COLL_COUNT(u.name) count
+SELECT STRICT_COUNT(u.name) count
 FROM FacebookUsers u;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
index 1b05c86..12eebf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
@@ -19,6 +19,6 @@
 
 USE TinySocial;
 
-SELECT COLL_COUNT(u.`friend-ids`) count
+SELECT STRICT_COUNT(u.`friend-ids`) count
 FROM FacebookUsers u
 ORDER BY u.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
index 2340771..e1a85e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
@@ -19,4 +19,4 @@
 
 USE TinySocial;
 
-COLL_COUNT(FacebookUsers);
+STRICT_COUNT(FacebookUsers);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
index 15af505..38e01dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
@@ -20,5 +20,5 @@
 USE TinySocial;
 
 WITH t AS 1
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`)), t
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`)), t
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
index c72c3c8..42d5f39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
@@ -19,5 +19,5 @@
 
 USE TinySocial;
 
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`))
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`))
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
index 56b042c..1274de2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
@@ -24,6 +24,6 @@
 GROUP BY e.deptno AS deptno
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM e AS i SELECT ELEMENT i.salary) ),
+  'avgpay': strict_avg( (FROM e AS i SELECT ELEMENT i.salary) ),
   'workers': (FROM e AS i SELECT ELEMENT  {'name': i.name, 'salary': i.salary})
-};
\ No newline at end of file
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
index dc7ac16..16714bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
@@ -24,6 +24,6 @@
 GROUP BY e.deptno AS deptno GROUP AS g(e AS e)
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM g AS i SELECT ELEMENT i.e.salary) ),
+  'avgpay': strict_avg( (FROM g AS i SELECT ELEMENT i.e.salary) ),
   'workers': (FROM g AS i SELECT ELEMENT  {'name': i.e.name, 'salary': i.e.salary})
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
index b752fe0..fde4de4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
@@ -25,6 +25,6 @@
 GROUP AS g(e as f)
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM f AS i SELECT ELEMENT i.salary) ),
+  'avgpay': strict_avg( (FROM f AS i SELECT ELEMENT i.salary) ),
   'workers': (FROM f AS i SELECT ELEMENT  {'name': i.name, 'salary': i.salary})
-};
\ No newline at end of file
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
index 70afc46..8b980e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
@@ -26,5 +26,5 @@
 GROUP AS eis(e AS e, i AS i, s AS s)
 SELECT ELEMENT {
       'deptId': deptId,
-      'star_cost': coll_sum( (FROM eis AS p SELECT ELEMENT p.e.salary + p.i.bonus) )
+      'star_cost': strict_sum( (FROM eis AS p SELECT ELEMENT p.e.salary + p.i.bonus) )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
index 8e6b987..1c9f321 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
@@ -26,7 +26,7 @@
 GROUP AS eis(e AS e, i AS i, s AS s)
 SELECT ELEMENT {
       'deptId': deptId,
-      'avgpay': coll_avg( (FROM eis AS g SELECT ELEMENT g.e.salary + g.i.bonus) ),
+      'avgpay': strict_avg( (FROM eis AS g SELECT ELEMENT g.e.salary + g.i.bonus) ),
       'topstar_details':
            (
                 FROM eis AS g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
index e647ff0..d762252 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
@@ -25,7 +25,7 @@
             GROUP BY c.tire_size AS tire_size GROUP AS g(c AS c, t AS t)
             SELECT ELEMENT {
                 'tire_size': tire_size,
-                'avg_total_price': coll_avg(
+                'avg_total_price': strict_avg(
                                          (  FROM g AS g
                                             SELECT ELEMENT g.c.price + 4 * g.t.price
                                           )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
index 8600077..ec47ba6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
@@ -23,6 +23,6 @@
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
 GROUP BY e.department_id GROUP AS g
-HAVING coll_count( ( SELECT VALUE g.e FROM g) )>0
+HAVING strict_count( ( SELECT VALUE g.e FROM g) )>0
 SELECT e.department_id as deptId, SUM(e.salary + i.bonus) AS star_cost;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
index 2bc83ef..0218637 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
@@ -19,9 +19,9 @@
 
 USE gby;
 
-// Tests that a SQL++ core aggregate function, e.g., COLL_AVG,
+// Tests that a SQL++ core aggregate function, e.g., STRICT_AVG,
 // cannot be used like a SQL-92 sugar, e.g., AVG. Its input must be a collection.
 FROM Employee e
 GROUP BY e.deptno AS deptno GROUP AS g
-SELECT deptno AS deptno, COLL_AVG(g.e.salary) AS avgpay,
+SELECT deptno AS deptno, STRICT_AVG(g.e.salary) AS avgpay,
        (SELECT i.e.name AS name, i.e.salary AS salary FROM g AS i) AS workers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
index 358eeda..7ea096f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count(( from g select value token ))}
+select element {'word':tok,'count':test.strict_count(( from g select value token ))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
index fb67e5b..0e52cdb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
@@ -28,7 +28,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
index 998dd43..972f0db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
@@ -27,7 +27,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
index b889ba9..eba03db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
@@ -27,7 +27,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
new file mode 100644
index 0000000..e2782c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description     : Test push down limit and select condition into the primary index lookup operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : bigint,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  index idx_LineItem_suppkey  on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
new file mode 100644
index 0000000..b8eac5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select element c
+from  LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
new file mode 100644
index 0000000..212479a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select element c
+from  LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
new file mode 100644
index 0000000..e9daaad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description     : Test push down limit into primary key lookup operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : bigint,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  index idx_LineItem_suppkey  on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
new file mode 100644
index 0000000..43f7d94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+explain select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
new file mode 100644
index 0000000..026aed7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
new file mode 100644
index 0000000..f214dc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description     : Test push down limit and select condition into primary index scan operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
new file mode 100644
index 0000000..2e0f056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp
new file mode 100644
index 0000000..657bfb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  where contains(dblpid, 'kimL89')
+  limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
new file mode 100644
index 0000000..6459b5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from  DBLP1 as paper
+where contains(dblpid, 'kimL89')
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
new file mode 100644
index 0000000..162cc35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description     : Test push down limit into the primary index scan operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
new file mode 100644
index 0000000..2e0f056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp
new file mode 100644
index 0000000..d2cb38a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
new file mode 100644
index 0000000..5b7c5be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from  DBLP1 as paper
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
index 5e73b39..ac09ccd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
@@ -25,10 +25,10 @@
 use test;
 
 
-select element coll_min(y)
+select element strict_min(y)
 from  [1,2] as x
 with  y as (
-      select element coll_min(i)
+      select element strict_min(i)
       from  [[1,2,3],[10,20,30],[-2,-5,0]] as i
   )
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
index 0954239..13e237b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
@@ -22,6 +22,6 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id, fu2.id
 LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
index a932f2a..1c6278c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
@@ -22,6 +22,6 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 LEFT OUTER JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))>2 AND STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id DESC, fu2.id DESC
 LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
index 087f7f6..e12399f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element l
     from  Customers as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
index 48a2456..2773865 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
@@ -23,12 +23,12 @@
 with  lonelyusers as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   ),
       lonelyusers2 as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   )
 select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
 from  lonelyusers as l1,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
new file mode 100644
index 0000000..c4d77ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Processed objects metrics on secondary index scan when it's an index-only plan
+ * Expected Res : Success
+ * Date         : 27 Apr 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.AddressType as
+{
+  number : bigint,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : bigint,
+  name : string,
+  age : bigint?,
+  address : AddressType?,
+  lastorder : {
+      oid : bigint,
+      total : float
+  }
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create index customer_name_idx on Customers(name);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp
new file mode 100644
index 0000000..6a5fd9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset Customers using localfs
+  ((`path`=`asterix_nc1://data/custord-tiny/customer-tiny-neg.adm`),
+  (`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp
new file mode 100644
index 0000000..881ed24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select count(*) from Customers
+where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
index 1c5126f..95d4681 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
@@ -24,5 +24,7 @@
 
 use test;
 
+set noindexonly 'true';
+
 select count(*) from Customers
 where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp
new file mode 100644
index 0000000..60229d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT
+  CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" as v1,
+  ( CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END ) LIKE "a%" as v2,
+  [ CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" ] as v3
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
index 075bfb8..5d87f1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
@@ -20,10 +20,10 @@
 use twitter;
 
 
-select element {'word':tok,'count':twitter.coll_count(( from g select value token ))}
+select element {'word':tok,'count':twitter.strict_count(( from g select value token ))}
 from  TwitterData as t,
       `word-tokens`(t.text) as token
 group by token as tok
 group as g
-order by coll_count(( from g select value token )) desc,tok
+order by strict_count(( from g select value token )) desc,tok
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp
new file mode 100644
index 0000000..bf35f87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": array_length((
+    from range(1, 6) t
+    select distinct value random()
+  )),
+
+  "t2": array_length((
+    from range(1, 6) t
+    select distinct value random(t)
+  )),
+
+  "t3": array_length((
+    from range(1, 6) t
+    select distinct value random(unix_time_from_datetime_in_ms(current_datetime()))
+  )),
+
+  "t4": array_length((
+    from [ tinyint("1"), smallint("2"), integer("3"), bigint("4"), float("5"), double("6") ] t
+    select distinct value random(t)
+  )),
+
+  "t5": [ random(missing) is missing, random(null) is null ]
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp
new file mode 100644
index 0000000..f650e0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+  id : bigint,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create type test.MyRecord as
+{
+  pid: uuid,
+  nested : MyRecordtmp?
+};
+
+create  dataset MyDatatmp(MyRecordtmp) primary key id;
+
+create  dataset MyData(MyRecord) primary key pid autogenerated;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp
new file mode 100644
index 0000000..8199dbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+load  dataset MyDatatmp using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`));
+
+insert into MyData
+select element {'nested':c}
+from  MyDatatmp as c
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp
new file mode 100644
index 0000000..5683fe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+create  index rtree_index_point  on MyData (nested.point) type rtree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp
new file mode 100644
index 0000000..93665a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+
+select element {'id':o.nested.id}
+from  MyData as o
+where test.`spatial-intersect`(o.nested.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..c4af7d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+  id : bigint,
+  point : point
+};
+
+create type test.MyRecord as
+{
+  nested : MyRecordtmp?
+};
+
+create  dataset MyData(MyRecord) primary key nested.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
index 81f2a3f..77e8d91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
@@ -41,5 +41,11 @@
   "21": 1 is valued,
   "22": null is not valued,
   "23": missing is not valued,
-  "24": 1 is not valued
+  "24": 1 is not valued,
+  "25": null is known,
+  "26": missing is known,
+  "27": 1 is known,
+  "28": null is not known,
+  "29": missing is not known,
+  "30": 1 is not known
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp
new file mode 100644
index 0000000..fe67901
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "e": trunc(e(), 2),
+  "pi": trunc(pi(), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp
new file mode 100644
index 0000000..80e002e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": tobigint(degrees(pi()))
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp
new file mode 100644
index 0000000..cca3dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{'result1':(smallint('2') div tinyint('+1')),'result2':(smallint('2') div smallint('2')),'result3':(smallint('2') div integer('+3')),'result4':(smallint('2') div bigint('-4')),'result5':(smallint('2') div float('-5.5f')),'result6':(smallint('2') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
new file mode 100644
index 0000000..7ce0f9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+{'result1':(integer('+3') div tinyint('+1')),'result2':(integer('+3') div smallint('2')),'result3':(integer('+3') div integer('+3')),'result4':(integer('+3') div bigint('-4')),'result5':(integer('+3') div float('-5.5f')),'result6':(integer('+3') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp
new file mode 100644
index 0000000..2f850c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{'result1':(bigint('-4') div tinyint('+1')),'result2':(bigint('-4') div smallint('2')),'result3':(bigint('-4') div integer('+3')),'result4':(bigint('-4') div bigint('-4')),'result5':(bigint('-4') div float('-5.5f')),'result6':(bigint('-4') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
new file mode 100644
index 0000000..5cf4b1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+{'result1':(tinyint('+1') div tinyint('+1')),'result2':(tinyint('+1') div smallint('2')),'result3':(tinyint('+1') div integer('+3')),'result4':(tinyint('+1') div bigint('-4')),'result5':(tinyint('+1') div float('-5.5f')),'result6':(tinyint('+1') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp
new file mode 100644
index 0000000..6d04878
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+{
+  "t1": trunc(radians(180), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 507e276..796913d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -118,6 +118,21 @@
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
+    <compilation-unit name="object_remove">
+      <output-dir compare="Text">object_remove</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_rename">
+      <output-dir compare="Text">object_rename</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_unwrap">
+      <output-dir compare="Text">object_unwrap</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
     <compilation-unit name="object_pairs">
       <output-dir compare="Text">object_pairs</output-dir>
     </compilation-unit>
@@ -141,18 +156,21 @@
     <compilation-unit name="closed-closed-fieldname-conflict_issue173">
       <output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
       <expected-error>Closed fields 0 and 1 have the same field name "name"</expected-error>
+      <source-location>false</source-location>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
     <compilation-unit name="open-closed-fieldname-conflict_issue173">
       <output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
       <expected-error>Open field "name" has the same field name as closed field at index 0</expected-error>
+      <source-location>false</source-location>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
     <compilation-unit name="open-open-fieldname-conflict_issue173">
       <output-dir compare="Text">open-open-fieldname-conflict_issue173</output-dir>
       <expected-error>Open fields 0 and 1 have the same field name "name"</expected-error>
+      <source-location>false</source-location>
     </compilation-unit>
   </test-case>
 </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp
new file mode 100644
index 0000000..b3dafba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp
new file mode 100644
index 0000000..2d03f92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
new file mode 100644
index 0000000..f7eabc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_remove(missing, missing) is missing,
+    object_remove(null, missing) is missing,
+    object_remove("{}", missing) is missing,
+    object_remove("{}", null) is null
+  ],
+  "t2": object_remove({"a":1, "b":2}, "a"),
+  "t3": object_remove({"a":1, "b":2}, "b"),
+  "t4": object_remove({"a":1, "b":2}, "c"),
+
+  /* open type */
+  "t5": (
+    select value object_remove(u, "lang")
+    from TwitterUsers as u
+    order by u.screen-name
+  ),
+
+  /* closed type */
+  "t6": (
+    select value object_remove(m, "user")
+    from TweetMessages as m
+    order by m.tweetid
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp
new file mode 100644
index 0000000..d6b47a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp
new file mode 100644
index 0000000..f6710f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp
new file mode 100644
index 0000000..5457dc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp
new file mode 100644
index 0000000..5f74fc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_rename(missing, missing, missing) is missing,
+    object_rename(null, missing, missing) is missing,
+    object_rename(null, null, missing) is missing,
+    object_rename(null, null, null) is null
+  ],
+  "t2": object_rename({"a":1, "b":2}, "b", "c"),
+  "t3": object_rename({"a":1, "b":2}, "z", "a"),
+
+  /* open type */
+  "t5": (
+    select value object_rename(u, "lang", "language")
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t6": (
+    select value object_rename(m, "user", "user_details")
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp
new file mode 100644
index 0000000..22bc025
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp
new file mode 100644
index 0000000..1aba6e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp
new file mode 100644
index 0000000..8f854e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
new file mode 100644
index 0000000..b79bd8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_unwrap(missing) is missing,
+    object_unwrap(null) is null,
+    object_unwrap("non-object") is null,
+    object_unwrap({"a":1, "b":2}) is null,
+    object_unwrap({}) is null
+  ],
+
+  "t2": object_unwrap({"a":"1"}),
+  "t3": object_unwrap({"a":{"b":"1"}}),
+  "t4": object_unwrap({"a":[1, 2]}),
+
+  "t5": (
+    select object_unwrap(o) from (
+      select lang
+      from TwitterUsers u
+      order by lang
+      limit 1
+    ) o
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
new file mode 100644
index 0000000..af175df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+drop dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
index 2b009ae..d1e30ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
@@ -28,7 +28,7 @@
 
 INSERT INTO ds1
 CASE
-  WHEN coll_count((
+  WHEN strict_count((
       select element x
       from  ds2 as x
       where (x.id = 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
index 341481a..f0782fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element [x.int_m, coll_count(( from g select value x ))]
+select element [x.int_m, strict_count(( from g select value x ))]
 from  DataOpen as x
 let id = x.id
 group by x.int_m
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
index a1c1923..c7b9b23 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
@@ -27,7 +27,7 @@
            FROM OrdersOpen o
            WHERE o.o_custkey >= -1
           )
-SELECT VALUE COLL_COUNT((
+SELECT VALUE STRICT_COUNT((
   SELECT VALUE l
   FROM l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
index 0e6f4e9..a1b3bac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
@@ -27,7 +27,7 @@
            FROM OrdersOpen o
            WHERE o.o_custkey >= -1
           )
-SELECT VALUE COLL_COUNT((
+SELECT VALUE STRICT_COUNT((
   SELECT VALUE l
   FROM l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
index f0fe4d0..e2f30a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
@@ -23,7 +23,7 @@
  * Date             : 5th July 2012
  */
 
-with  a as [every x in [1,2] satisfies (coll_avg([x,1]) = 1),every x in ['1','2'] satisfies (string(x) = '1'),every x in ['1','2'] satisfies (`string-length`(x) = 1),every x in [[1,2],[10],[1,5,7,8]] satisfies (coll_count(x) = 1),every x in [[2],[10],[8]] satisfies (coll_count(x) = 1),every x in [true,false] satisfies boolean('true'),every x in [true,true] satisfies not(x),every x in [1,2,3],
+with  a as [every x in [1,2] satisfies (strict_avg([x,1]) = 1),every x in ['1','2'] satisfies (string(x) = '1'),every x in ['1','2'] satisfies (`string-length`(x) = 1),every x in [[1,2],[10],[1,5,7,8]] satisfies (strict_count(x) = 1),every x in [[2],[10],[8]] satisfies (strict_count(x) = 1),every x in [true,false] satisfies boolean('true'),every x in [true,true] satisfies not(x),every x in [1,2,3],
 y in [4,5,6] satisfies ((x + y) = 5),every x in [1,2,3],
 y in [4,5,6] satisfies ((x - y) = 5),every x in [1,2,3],
 y in [4,5,6] satisfies ((x * y) = 10),every x in ['ab','cd'],
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
index 5274bb5..8975aee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date            : 6th July 2012
  */
 
-with  a as [some x in [1,2] satisfies ((x + x) = 3),some x in [1,2] satisfies ((x + x) = 2),some x in [1,2] satisfies ((x - 2) = 2),some x in [1,2] satisfies ((x - 2) = 0),some x in [1,2] satisfies ((x * 2) = 4),some x in [1,2] satisfies ((x / 2) = 1),some x in [1,2] satisfies (coll_avg([x,1]) = 1),some x in [1,2] satisfies boolean('true'),some x in [1,2] satisfies boolean('false'),some x in [true,false] satisfies not(x),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2))]
+with  a as [some x in [1,2] satisfies ((x + x) = 3),some x in [1,2] satisfies ((x + x) = 2),some x in [1,2] satisfies ((x - 2) = 2),some x in [1,2] satisfies ((x - 2) = 0),some x in [1,2] satisfies ((x * 2) = 4),some x in [1,2] satisfies ((x / 2) = 1),some x in [1,2] satisfies (strict_avg([x,1]) = 1),some x in [1,2] satisfies boolean('true'),some x in [1,2] satisfies boolean('false'),some x in [true,false] satisfies not(x),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2))]
 select element i
 from  a as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
index 562ff36..0212bb4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
@@ -24,7 +24,7 @@
  * Date             : 5th July 2012
  */
 
-with  a as [some x in ['foo','foobar','foot','fox'] satisfies (`string-length`(x) = 3),some x in [[5,4,3,2],[1,2,3,4,5,6,7,8],[4,2,3,4]] satisfies (coll_count(x) = 8),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2)),some x in ['A','B','C'] satisfies (x = 'A'),some x in [1,2,3],
+with  a as [some x in ['foo','foobar','foot','fox'] satisfies (`string-length`(x) = 3),some x in [[5,4,3,2],[1,2,3,4,5,6,7,8],[4,2,3,4]] satisfies (strict_count(x) = 8),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2)),some x in ['A','B','C'] satisfies (x = 'A'),some x in [1,2,3],
 y in [4,5,6] satisfies ((x + y) = 5),some x in [1,2,3],
 y in [4,5,6] satisfies ((x - y) = 5),some x in [1,2,3],
 y in [4,5,6] satisfies ((x * y) = 10),some x in [1,2,3],
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
index 7a44bfe..62d2437 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
@@ -19,4 +19,4 @@
 
 use test1;
 
-coll_count(test.samptable);
+strict_count(test.samptable);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
index 9705de5..4238881 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'custage':age,'count': coll_count(( from g select value c))}
+select element {'custage':age,'count': strict_count(( from g select value c))}
 from  Customers as c
 group by c.age as age
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
index 63442c4..da03b3b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
@@ -27,6 +27,6 @@
 where (`spatial-intersect`(t.loc,region) and (t.time > datetime('2011-05-15T00:00:00Z')) and (t.time < datetime('2011-05-16T23:59:59Z')) and contains(t.text,keywords))
 group by `spatial-cell`(t.loc,`create-point`(24.5,-125.5),3.0,3.0) as c
 group as g
-let num = coll_count((from g select value t))
+let num = strict_count((from g select value t))
 order by num
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
index eac020a..179fa9d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
@@ -25,7 +25,7 @@
       from  MyData as o
       group by `spatial-cell`(o.loc, `create-point`(0.0,0.0),5.0,5.0) as c
       group as g
-      let  num = coll_count(( from g select value o ))
+      let  num = strict_count(( from g select value o ))
       order by num
   )
 select element g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp
new file mode 100644
index 0000000..e127372
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ {
+   "t1": [ reverse(missing) is missing, reverse(null) is null ],
+   "t2": reverse(""),
+   "t3": reverse("abcd"),
+   "t4": string_to_codepoint(reverse("a\u00D7\u2103\u00F7\u2109b")),
+   "t5": ( from ["ab", "abc", "abcd"] t select value reverse(t) order by t )
+ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
index b9c79d0..d39e51e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
@@ -23,4 +23,4 @@
  * Date           : 18th April 2012
  */
 
-{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2 - 1))};
+{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-6),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') div 2 - 1))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
index 860173e..e1ef68b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
@@ -24,5 +24,15 @@
  */
 
 select element a
-from  [substring('hello world',6,5),substring('hello world',0,11),substring('hello world',2,9),substring('ABCD',2,2),substring('ABCD',0,4),substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',0,`string-length`('UC Irvine')),substring(substring('UC Irvine',3),0,`string-length`('Irvine'))] as a
+from [
+    substring('hello world',6,5),
+    substring('hello world',0,11),
+    substring('hello world',2,9),
+    substring('ABCD',2,2),
+    substring('ABCD',0,4),
+    substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),
+    substring('UC Irvine',0,`string-length`('UC Irvine')),
+    substring(substring('UC Irvine',3),0,`string-length`('Irvine'))
+  ] as a
+
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
index 8ea572c..ae943d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{'result1':test.substring('HEllow',-1)};
+{'result1':test.substring('HEllow',-3)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
index 45f924a..cf6b506 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
@@ -22,7 +22,7 @@
 /** Finds customers whose orders have all possible priorities.*/
 
 WITH priorities AS
-COLL_COUNT((
+STRICT_COUNT((
   SELECT DISTINCT o.o_orderpriority FROM Orders o
 ))
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
index b67806d..f7f3661 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
@@ -20,19 +20,19 @@
 use test;
 
 
-{'m0':test.coll_max((
+{'m0':test.strict_max((
     select element i.time
     from  tsdata as i
-)),'m1':test.coll_max((
+)),'m1':test.strict_max((
     select element i.date
     from  tsdata as i
-)),'m2':test.coll_max((
+)),'m2':test.strict_max((
     select element i.datetime
     from  tsdata as i
-)),'m3':test.coll_max((
+)),'m3':test.strict_max((
     select element i.dtduration
     from  tsdata as i
-)),'m4':test.coll_max((
+)),'m4':test.strict_max((
     select element i.ymduration
     from  tsdata as i
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
index cfc6dd7..28ca219 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
@@ -20,19 +20,19 @@
 use test;
 
 
-{'m0':test.coll_min((
+{'m0':test.strict_min((
     select element i.time
     from  tsdata as i
-)),'m1':test.coll_min((
+)),'m1':test.strict_min((
     select element i.date
     from  tsdata as i
-)),'m2':test.coll_min((
+)),'m2':test.strict_min((
     select element i.datetime
     from  tsdata as i
-)),'m3':test.coll_min((
+)),'m3':test.strict_min((
     select element i.dtduration
     from  tsdata as i
-)),'m4':test.coll_min((
+)),'m4':test.strict_min((
     select element i.ymduration
     from  tsdata as i
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
index 0f9b329..4cab752 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
@@ -25,8 +25,8 @@
 
 
 select element {'timebin':bin,
-    'count':coll_count((select value 1 from g)),
-    'total_ms':coll_sum((
+    'count':strict_count((select value 1 from g)),
+    'total_ms':strict_sum((
         select element
         `ms_from_day_time_duration`(`duration-from-interval`(`get-overlapping-interval`(bin, g.i2.interval)))
         from g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
index 9a29efb..7fc4ccd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
@@ -27,10 +27,10 @@
         group by i.app as subgid
         order by subgid,multitask.count(i)
     )}
-from  multitask.`overlap-bins`(multitask.interval(multitask.coll_min((
+from  multitask.`overlap-bins`(multitask.interval(multitask.strict_min((
     select element i.time
     from  logs as i
-)),multitask.coll_max((
+)),multitask.strict_max((
     select element (i.time + multitask.`duration-from-ms`((i.duration * 1000)))
     from  logs as i
 ))),multitask.time('00:00:00.000'),multitask.`day_time_duration`('PT1M')) as bin
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
index 85a66f2..5952910 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
@@ -21,7 +21,7 @@
 
 USE TinySocial;
 
-SELECT ELEMENT coll_avg((
+SELECT ELEMENT strict_avg((
 select element LENGTH(message.message)
 FROM FacebookMessages AS message
 WHERE message.`in-response-to` >= 1 and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
index 7ca10bd..de80b98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element fbu
     from  FacebookUsers as fbu
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
index ac378d0..468c774 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element t
     from  TweetMessages as t
     where (t.tweetid = '13')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
index 7ca10bd..de80b98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element fbu
     from  FacebookUsers as fbu
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
index ac378d0..468c774 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element t
     from  TweetMessages as t
     where (t.tweetid = '13')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
index fa84678..3ec28e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
@@ -19,7 +19,7 @@
 
 USE TinySocial;
 
-SELECT ELEMENT coll_avg((
+SELECT ELEMENT strict_avg((
 select element LENGTH(message.message)
 FROM FacebookMessages AS message
 WHERE message.`in-response-to` >= 1 and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
index 455eab7..b7fb014 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
@@ -20,7 +20,7 @@
 
 USE tpcds;
 
-SELECT VALUE {"h8_30_to_9" : COLL_COUNT((select 1
+SELECT VALUE {"h8_30_to_9" : STRICT_COUNT((select 1
      FROM store_sales ss1, household_demographics , time_dim, store s1
      WHERE ss1.ss_sold_time_sk = time_dim.t_time_sk
      AND ss1.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -30,7 +30,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s1.s_store_name = 'ese')), "h9_to_9_30": COLL_COUNT((select 1
+     AND s1.s_store_name = 'ese')), "h9_to_9_30": STRICT_COUNT((select 1
      FROM store_sales ss2, household_demographics , time_dim, store s2
      WHERE ss2.ss_sold_time_sk = time_dim.t_time_sk
      AND ss2.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -40,7 +40,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s2.s_store_name = 'ese')), "h9_30_to_10": COLL_COUNT((select 1
+     AND s2.s_store_name = 'ese')), "h9_30_to_10": STRICT_COUNT((select 1
      FROM store_sales ss3, household_demographics , time_dim, store s3
      WHERE ss3.ss_sold_time_sk = time_dim.t_time_sk
      AND ss3.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -50,7 +50,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s3.s_store_name = 'ese')), "h10_to_10_30": COLL_COUNT((select 1
+     AND s3.s_store_name = 'ese')), "h10_to_10_30": STRICT_COUNT((select 1
      FROM store_sales ss4, household_demographics , time_dim, store s4
      WHERE ss4.ss_sold_time_sk = time_dim.t_time_sk
      AND ss4.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -60,7 +60,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s4.s_store_name = 'ese')), "h10_30_to_11": COLL_COUNT( (select 1
+     AND s4.s_store_name = 'ese')), "h10_30_to_11": STRICT_COUNT( (select 1
      FROM store_sales ss5, household_demographics , time_dim, store s5
      WHERE ss5.ss_sold_time_sk = time_dim.t_time_sk
      AND ss5.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -70,7 +70,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s5.s_store_name = 'ese')), "h11_to_11_30": COLL_COUNT((select 1
+     AND s5.s_store_name = 'ese')), "h11_to_11_30": STRICT_COUNT((select 1
      FROM store_sales ss6, household_demographics , time_dim, store s6
      WHERE ss6.ss_sold_time_sk = time_dim.t_time_sk
      AND ss6.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -80,7 +80,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s6.s_store_name = 'ese')), "h11_30_to_12": COLL_COUNT((select 1
+     AND s6.s_store_name = 'ese')), "h11_30_to_12": STRICT_COUNT((select 1
      FROM store_sales ss7, household_demographics , time_dim, store s7
      WHERE ss7.ss_sold_time_sk = time_dim.t_time_sk
      AND ss7.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -90,7 +90,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s7.s_store_name = 'ese')), "h12_to_12_30":  COLL_COUNT((select 1
+     AND s7.s_store_name = 'ese')), "h12_to_12_30":  STRICT_COUNT((select 1
      FROM store_sales ss8, household_demographics , time_dim, store s8
      WHERE ss8.ss_sold_time_sk = time_dim.t_time_sk
      AND ss8.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -100,4 +100,4 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s8.s_store_name = 'ese'))};
\ No newline at end of file
+     AND s8.s_store_name = 'ese'))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
index c13f668..c1943e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
@@ -30,6 +30,6 @@
     and s.ss_sold_date_sk = d.d_date_sk
     and s.ss_item_sk = i.i_item_sk
 group by a.ca_state group as stategrp
-having coll_count((select value a from stategrp)) >= 10
+having strict_count((select value a from stategrp)) >= 10
 order by cnt
 limit 100;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
index 3a1b5c2..fadfa8d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
@@ -36,7 +36,7 @@
  group by item.i_item_id
 )
 
-select coll_count((
+select strict_count((
  select *
  from tab t1, tab t2
-));
\ No newline at end of file
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index 3aea572..766e704 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(tpch.coll_sum((
+select element {'year':year,'mkt_share':(tpch.strict_sum((
           select element case when i.t.s_name = 'BRAZIL' then i.t.revenue else 0.0 end
           from  g as i
-      )) / tpch.coll_sum((
+      )) / tpch.strict_sum((
           select element i.t.revenue
           from  g as i
       )))}
@@ -67,4 +67,4 @@
 group by t.year as year
 group as g
 order by year
-;
\ No newline at end of file
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index 6dfd5af..a9116b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':COLL_SUM((
+select element {'nation':nation,'o_year':o_year,'sum_profit':STRICT_SUM((
         select element pr.profit.amount
         from  g as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
index 6038d15..5066186 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value locn) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index 3a6ac4a..8730e73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value locn) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
index a98ec6e..bbf5464 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element {'l_shipmode':l_shipmode,'high_line_count':tpch.coll_sum((
+select element {'l_shipmode':l_shipmode,'high_line_count':tpch.strict_sum((
         select element case i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH'
                        when true then 1
                        when false then 0
                        end
         from  g as i
-    )),'low_line_count':tpch.coll_sum((
+    )),'low_line_count':tpch.strict_sum((
         select element case i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH'
                        when true then 0
                        when false then 1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index a15ab14..860bce0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
@@ -41,6 +41,6 @@
 ) as gco
 group by gco.c_count as c_count
 group as g
-let  custdist = coll_count(( select value gco from g ))
+let  custdist = strict_count(( select value gco from g ))
 order by custdist desc,c_count desc
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 2ec6a9c..651c398 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element (100.0 * COLL_SUM((
+select element (100.0 * STRICT_SUM((
       select element case i.p_type like 'PROMO%' when true then i.l_extendedprice * (1 - i.l_discount) else 0.0 end
       from (from g select value lp) as i
-  )) / COLL_SUM((
+  )) / STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value lp) as i
   )))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
index 46a278c..2bf53b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element g.profit.amount
         from  g
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index f8211c9..9052e05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey, sum(o_orderkey_count) AS c_count
         FROM  (
                 SELECT c_custkey,
-                       coll_count(
+                       strict_count(
                            (
                             select element o_orderkey
                             from  Orders
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
index c6afe18..352b7d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
@@ -37,7 +37,7 @@
 
 SELECT cntrycode, count(ct) AS numcust, sum(c_acctbal) AS totacctbal
 FROM  q22_customer_tmp as ct
-WHERE coll_count((
+WHERE strict_count((
                 SELECT ELEMENT Orders
                 FROM  Orders
                 WHERE ct.c_custkey = o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
index fc9c64d..b1f35f3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -28,7 +28,7 @@
 
 SELECT l_returnflag,
        l_linestatus,
-       coll_count(cheaps) AS count_cheaps,
+       strict_count(cheaps) AS count_cheaps,
        total_charges AS total_charges
 FROM  LineItem as l
 WHERE l_shipdate <= '1998-09-02'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
index 5186119..b8acf3b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT  l_returnflag,
         l_linestatus,
-        coll_count(cheaps) AS count_cheaps,
-        coll_avg(expensives) AS avg_expensive_discounts,
+        strict_count(cheaps) AS count_cheaps,
+        strict_avg(expensives) AS avg_expensive_discounts,
         sum_disc_prices AS sum_disc_prices,
         total_charges AS total_charges
 FROM  LineItem AS l
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
index cb1b00f8..1b959d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT l_returnflag,
        l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 WHERE l_shipdate <= '1998-09-02'
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index fb3e8e7..597b054 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey AS c_custkey, sum(co.o_orderkey_count) AS c_count
         FROM  (
                 SELECT c.c_custkey AS c_custkey,
-                       coll_count(
+                       strict_count(
                            (
                             select element o.o_orderkey
                             from  Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
index 531e17f..f4a683b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
@@ -38,7 +38,7 @@
 
 SELECT cntrycode AS cntrycode, count(ct) AS numcust, sum(ct.c_acctbal) AS totacctbal
 FROM  q22_customer_tmp() as ct
-WHERE coll_count((
+WHERE strict_count((
                 SELECT ELEMENT o
                 FROM  Orders AS o
                 WHERE ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
index bfd62c9..f193452 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -28,7 +28,7 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheaps) AS count_cheaps,
+       strict_count(cheaps) AS count_cheaps,
        total_charges AS total_charges
 FROM  LineItem as l
 WHERE l.l_shipdate <= '1998-09-02'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
index 3ee3a31..7beda71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT  l_returnflag AS l_returnflag,
         l_linestatus AS l_linestatus,
-        coll_count(cheaps) AS count_cheaps,
-        coll_avg(expensives) AS avg_expensive_discounts,
+        strict_count(cheaps) AS count_cheaps,
+        strict_avg(expensives) AS avg_expensive_discounts,
         sum_disc_prices AS sum_disc_prices,
         total_charges AS total_charges
 FROM  LineItem AS l
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
index 17e37b3..2f49c59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 WHERE l.l_shipdate <= '1998-09-02'
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
index 389a35d..591d336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
index e137e9f9..8633039 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
@@ -33,7 +33,7 @@
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = tpch.coll_sum((
+        let  sum = tpch.strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
index b960393..09ec48d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-{'revenue':COLL_SUM((
+{'revenue':STRICT_SUM((
     select element (l.l_extendedprice * l.l_discount)
     from  LineItem as l
     where ((l.l_shipdate >= '1994-01-01') and (l.l_shipdate < '1995-01-01') and (l.l_discount >= 0.05) and (l.l_discount <= 0.07) and (l.l_quantity < 24))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index ce12ffd..61c0b7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(COLL_SUM((
+select element {'year':year,'mkt_share':(STRICT_SUM((
           select element case i.s_name = 'BRAZIL' when true then i.revenue when false then 0.0 end
           from  (from g select value t) as i
-      )) / COLL_SUM((
+      )) / STRICT_SUM((
           select element i.revenue
           from  (from g select value t) as i
       )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index f469707..a31e791 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element pr.amount
         from (from g select value profit) as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
index b5390d4..d56543e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element {'l_shipmode':l_shipmode,'high_line_count':tpch.coll_sum((
+select element {'l_shipmode':l_shipmode,'high_line_count':tpch.strict_sum((
         select element case
                         when i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH' then 1
                         else 0
                         end
         from  g as i
-    )),'low_line_count':tpch.coll_sum((
+    )),'low_line_count':tpch.strict_sum((
         select element case
                          when i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH' then 0
                          else 1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index 0c0bdaa..9df4e16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 864793d..b715c25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element (100.0 * tpch.coll_sum((
+select element (100.0 * tpch.strict_sum((
       select element case
                        when i.lp.p_type like 'PROMO%' then i.lp.l_extendedprice * (1 - i.lp.l_discount)
                        else 0.0
                      end
       from  g as i
-  )) / tpch.coll_sum((
+  )) / tpch.strict_sum((
       select element (i.lp.l_extendedprice * (1 - i.lp.l_discount))
       from  g as i
   )))
@@ -34,4 +34,4 @@
 let lp = {'p_type': p.p_type, 'l_extendedprice': l.l_extendedprice, 'l_discount': l.l_discount}
 where ((l.l_partkey = p.p_partkey) and (l.l_shipdate >= '1995-09-01') and (l.l_shipdate < '1995-10-01'))
 group by 1 as t group as g(lp as lp)
-;
\ No newline at end of file
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
index 4cd5d54..d542ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
@@ -34,18 +34,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.strict_sum((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (coll_count((
+where (strict_count((
     select element o
     from  Orders as o
     where ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
index 38ba16f..6888d5f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
index 555437a..33618d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
index 6877db8..3ef97a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheap),'count_expensives':coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheap),'count_expensives':strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
index aefe8a3..7fac9f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
@@ -26,30 +26,30 @@
 use tpch;
 
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':coll_count((
+)),'count_order':strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
index c488ab1..c8dd6f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
@@ -26,10 +26,10 @@
 use tpch;
 
 
-{'count_cheaps': coll_count((
+{'count_cheaps': strict_count((
     select element l.l_quantity
     from  LineItem as l
-)),'count_expensives': coll_sum((
+)),'count_expensives': strict_sum((
     select element e
     from  (
         select element l.l_extendedprice
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
index 389a35d..591d336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
index e137e9f9..8633039 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
@@ -33,7 +33,7 @@
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = tpch.coll_sum((
+        let  sum = tpch.strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
index b960393..09ec48d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-{'revenue':COLL_SUM((
+{'revenue':STRICT_SUM((
     select element (l.l_extendedprice * l.l_discount)
     from  LineItem as l
     where ((l.l_shipdate >= '1994-01-01') and (l.l_shipdate < '1995-01-01') and (l.l_discount >= 0.05) and (l.l_discount <= 0.07) and (l.l_quantity < 24))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index ce12ffd..61c0b7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(COLL_SUM((
+select element {'year':year,'mkt_share':(STRICT_SUM((
           select element case i.s_name = 'BRAZIL' when true then i.revenue when false then 0.0 end
           from  (from g select value t) as i
-      )) / COLL_SUM((
+      )) / STRICT_SUM((
           select element i.revenue
           from  (from g select value t) as i
       )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index f469707..a31e791 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element pr.amount
         from (from g select value profit) as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
index fa54051..f88a2ba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
@@ -20,11 +20,11 @@
 use tpch;
 
 
-select element {'l_shipmode':l_shipmode,'high_line_count':COLL_SUM((
+select element {'l_shipmode':l_shipmode,'high_line_count':STRICT_SUM((
         select element
             CASE i.o_orderpriority = '1-URGENT' OR i.o_orderpriority = '2-HIGH' WHEN true THEN 1 WHEN false THEN 0 END
         from  (select value o from g) as i
-    )),'low_line_count':COLL_SUM((
+    )),'low_line_count':STRICT_SUM((
         select element case when i.o_orderpriority = '1-URGENT' OR i.o_orderpriority = '2-HIGH' then 0 else 1 end
         from (select value o from g) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index 0c0bdaa..9df4e16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 2ec6a9c..651c398 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element (100.0 * COLL_SUM((
+select element (100.0 * STRICT_SUM((
       select element case i.p_type like 'PROMO%' when true then i.l_extendedprice * (1 - i.l_discount) else 0.0 end
       from (from g select value lp) as i
-  )) / COLL_SUM((
+  )) / STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value lp) as i
   )))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
index 4cd5d54..d542ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
@@ -34,18 +34,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.strict_sum((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (coll_count((
+where (strict_count((
     select element o
     from  Orders as o
     where ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
index 46a278c..2bf53b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element g.profit.amount
         from  g
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
index 38ba16f..6888d5f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
index f25f99d..4c25891 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
index 6877db8..3ef97a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheap),'count_expensives':coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheap),'count_expensives':strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
index aefe8a3..7fac9f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
@@ -26,30 +26,30 @@
 use tpch;
 
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':coll_count((
+)),'count_order':strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
index c488ab1..c8dd6f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
@@ -26,10 +26,10 @@
 use tpch;
 
 
-{'count_cheaps': coll_count((
+{'count_cheaps': strict_count((
     select element l.l_quantity
     from  LineItem as l
-)),'count_expensives': coll_sum((
+)),'count_expensives': strict_sum((
     select element e
     from  (
         select element l.l_extendedprice
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp
new file mode 100644
index 0000000..d3162f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type sub as {
+  subscriptionId: uuid
+};
+
+create dataset subscriptions(sub) primary key subscriptionId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp
new file mode 100644
index 0000000..c4ba876
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp
new file mode 100644
index 0000000..0677150
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker3","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":"dataverse1",
+  "brokerName":"broker3","created":current_datetime()}]}
+END
+))
+) returning
+(let subs = record.brokerSubscriptions
+select value brokerName from subs
+order by created DESC limit 1)[0];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp
new file mode 100644
index 0000000..51889d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+select value array_count(brokerSubscriptions) from subscriptions;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
index a383042..709b3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
@@ -23,7 +23,7 @@
  * Date         : 31st May 2013
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  `Metadata`.`Function` as x
     where (x.DataverseName = 'test')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
index a383042..709b3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
@@ -23,7 +23,7 @@
  * Date         : 31st May 2013
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  `Metadata`.`Function` as x
     where (x.DataverseName = 'test')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
index 1380ed2..efe17a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
@@ -17,8 +17,4 @@
  * under the License.
  */
 use test;
-
-
 select element test.computeBonus(-1,-1);
-select element test.computeBonus(1,-1);
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp
new file mode 100644
index 0000000..5ee25ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use test;
+select element test.computeBonus(1,-1);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 0e619cc..cc4c573 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -37,7 +37,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index b3a3eed..dd7dfb7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -37,7 +37,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 75e879c..71200c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -37,7 +37,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_nc_threaddump_1/cluster_state_cc_threaddump_1.1.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_nc_threaddump_1/cluster_state_cc_threaddump_1.1.regex
index 3f1bfdc..9adcd7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_nc_threaddump_1/cluster_state_cc_threaddump_1.1.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_nc_threaddump_1/cluster_state_cc_threaddump_1.1.regex
@@ -4,7 +4,7 @@
 /"lock_name" : ".*"/
 /"name" : ".*"/
 /"stack" : \[/
-/"java.lang.Thread.run\(Thread.java:[0-9]+\)"/
+/".*java.lang.Thread.run\(Thread.java:[0-9]+\)"/
 /"state" : "TIMED_WAITING"/
 /"state" : "RUNNABLE"/
 /"state" : "WAITING"/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
new file mode 100644
index 0000000..fca4cbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "MISSING" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "MISSING" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "MISSING" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "MISSING" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "MISSING" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "MISSING" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
new file mode 100644
index 0000000..cfcc525
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NaN" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "NaN" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "NaN" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "NaN" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NaN" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NaN" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
new file mode 100644
index 0000000..1b75b97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "-INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "-INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "-INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "-INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "-INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "-INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
new file mode 100644
index 0000000..5c53877
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NULL" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "NULL" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "NULL" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "NULL" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NULL" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NULL" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
new file mode 100644
index 0000000..ebc67b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm
new file mode 100644
index 0000000..519d3ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm
@@ -0,0 +1 @@
+{ "s_suppkey": 1, "s_name": "Supplier#000000001", "s_address": " N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ", "s_nationkey": 17, "s_phone": "27-918-335-1736", "s_acctbal": 5755.94, "s_comment": "each slyly above the careful" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm
new file mode 100644
index 0000000..dda08f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm
@@ -0,0 +1,5 @@
+{ "s_suppkey": 1, "s_name": "Supplier#000000001", "s_address": " N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ", "s_nationkey": 17, "s_phone": "27-918-335-1736", "s_acctbal": 5755.94, "s_comment": "each slyly above the careful" }
+{ "s_suppkey": 2, "s_name": "Supplier#000000002", "s_address": "89eJ5ksX3ImxJQBvxObC,", "s_nationkey": 5, "s_phone": "15-679-861-2259", "s_acctbal": 4032.68, "s_comment": " slyly bold instructions. idle dependen" }
+{ "s_suppkey": 3, "s_name": "Supplier#000000003", "s_address": "q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3", "s_nationkey": 1, "s_phone": "11-383-516-1199", "s_acctbal": 4192.4, "s_comment": "blithely silent requests after the express dependencies are sl" }
+{ "s_suppkey": 4, "s_name": "Supplier#000000004", "s_address": "Bk7ah4CK8SYQTepEmvMkkgMwg", "s_nationkey": 15, "s_phone": "25-843-787-7479", "s_acctbal": 4641.08, "s_comment": "riously even requests above the exp" }
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm
new file mode 100644
index 0000000..69756c0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm
@@ -0,0 +1 @@
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm
new file mode 100644
index 0000000..49ed112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm
@@ -0,0 +1,5 @@
+{ "s_suppkey": 2, "s_name": "Supplier#000000002", "s_address": "89eJ5ksX3ImxJQBvxObC,", "s_nationkey": 5, "s_phone": "15-679-861-2259", "s_acctbal": 4032.68, "s_comment": " slyly bold instructions. idle dependen" }
+{ "s_suppkey": 3, "s_name": "Supplier#000000003", "s_address": "q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3", "s_nationkey": 1, "s_phone": "11-383-516-1199", "s_acctbal": 4192.4, "s_comment": "blithely silent requests after the express dependencies are sl" }
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
+{ "s_suppkey": 6, "s_name": "Supplier#000000006", "s_address": "tQxuVm7s7CnK", "s_nationkey": 14, "s_phone": "24-696-997-4969", "s_acctbal": 1365.79, "s_comment": "final accounts. regular dolphins use against the furiously ironic decoys. " }
+{ "s_suppkey": 10, "s_name": "Supplier#000000010", "s_address": "Saygah3gYWMp72i PY", "s_nationkey": 24, "s_phone": "34-852-489-8585", "s_acctbal": 3891.91, "s_comment": "ing waters. regular requests ar" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
index 58d9bdc..dd41cbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
@@ -1,3 +1,3 @@
-{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20, "-cashBack": -100 } }
-{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70, "-cashBack": -350 } }
-{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90, "-cashBack": -450 } }
+{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20.0, "-cashBack": -100 } }
+{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70.0, "-cashBack": -350 } }
+{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90.0, "-cashBack": -450 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
index 89f8083..2990e0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
@@ -1,3 +1,3 @@
 { "custname": "Jodi Rotruck", "age": null, "MathAge": { "age": null, "age+5": null, "age-5": null, "age*5": null, "age/5": null, "-age": null } }
-{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3, "-age": -19 } }
-{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2, "-age": -12 } }
+{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3.8, "-age": -19 } }
+{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2.4, "-age": -12 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
index 98cedcc..cbbd46b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
@@ -1,44 +1,44 @@
-distribute result [$$36]
+distribute result [$$38]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$36])
+    project ([$$38])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"deptId": $#1, "star_cost": $$39}]
+      assign [$$38] <- [{"deptId": $#1, "star_cost": $$41}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$44]) decor ([]) {
-                    aggregate [$$39] <- [agg-sql-sum($$43)]
+          group by ([$#1 := $$46]) decor ([]) {
+                    aggregate [$$41] <- [agg-sql-sum($$45)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$44]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$46]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
-              group by ([$$44 := $$37]) decor ([]) {
-                        aggregate [$$43] <- [agg-local-sql-sum($$34)]
+            -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+              group by ([$$46 := $$39]) decor ([]) {
+                        aggregate [$$45] <- [agg-local-sql-sum($$36)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$37]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$39]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$34, $$37])
+                  project ([$$36, $$39])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$37] <- [substring($$41.getField("department_id"), 0)]
+                    assign [$$39] <- [substring($$43.getField("department_id"), 0)]
                     -- ASSIGN  |PARTITIONED|
-                      project ([$$34, $$41])
+                      project ([$$36, $$43])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$34, $$41] <- [$$e.getField("salary"), $$e.getField("dept")]
+                        assign [$$36, $$43] <- [$$e.getField("salary"), $$e.getField("dept")]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$e])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$38, $$e] <- gby.Employee
+                              data-scan []<-[$$40, $$e] <- gby.Employee
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
index e2e1c91..7aec0e1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
@@ -1,40 +1,40 @@
-distribute result [$$35]
+distribute result [$$37]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$35])
+    project ([$$37])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$35] <- [{"deptId": $#1, "star_cost": $$38}]
+      assign [$$37] <- [{"deptId": $#1, "star_cost": $$40}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$42]) decor ([]) {
-                    aggregate [$$38] <- [agg-sql-sum($$41)]
+          group by ([$#1 := $$44]) decor ([]) {
+                    aggregate [$$40] <- [agg-sql-sum($$43)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$42]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$44]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-              group by ([$$42 := $$36]) decor ([]) {
-                        aggregate [$$41] <- [agg-local-sql-sum($$33)]
+            -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+              group by ([$$44 := $$38]) decor ([]) {
+                        aggregate [$$43] <- [agg-local-sql-sum($$35)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$36]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$38]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$33, $$36])
+                  project ([$$35, $$38])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$36, $$33] <- [substring($$e.getField(1), 0), $$e.getField(2)]
+                    assign [$$38, $$35] <- [substring($$e.getField(1), 0), $$e.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$37, $$e] <- gby.Employee
+                          data-scan []<-[$$39, $$e] <- gby.Employee
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
index 51fdf04..f599e28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
@@ -1,3 +1 @@
 10
-10
-10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
new file mode 100644
index 0000000..022f6a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
@@ -0,0 +1 @@
+{ "userName": "c1121u1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm
new file mode 100644
index 0000000..83efad9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm
@@ -0,0 +1 @@
+{ "t": { "id": 999403843673718784, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393280, "id_str": "988455822727393280", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } }, "temp": [ { "f": { "id": 999403845, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393281, "id_str": "988455822727393281", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } } }, { "f": { "id": 999403846, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393282, "id_str": "988455822727393282", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
new file mode 100644
index 0000000..2a16ab1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
@@ -0,0 +1,14 @@
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
new file mode 100644
index 0000000..b9029e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
@@ -0,0 +1,3 @@
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-69.1991349,-12.6006222],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-69.199136,-12.6010968],[-69.1972972,-12.6010968],[-69.1972972,-12.5998133],[-69.199136,-12.5998133],[-69.199136,-12.6010968]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[1,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[1,2],[7,2],[7,8],[1,8],[1,2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-113.981,39.195],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-113.98,39.198],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-113.981,39.195],[-113.98,39.195],[-113.98,39.198],[-113.981,39.198],[-113.981,39.195]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
new file mode 100644
index 0000000..048a6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
@@ -0,0 +1,14 @@
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
new file mode 100644
index 0000000..3af2594
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
@@ -0,0 +1,14 @@
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 13, "XMax": -69.1972972, "XMin": -69.199136, "YMax": -12.5998133, "YMin": -12.6010968, "Binary": hex("01020000000D00000081BF4EA0BE4C51C0CE80C4C0843329C033DDEBA4BE4C51C0D388997D1E3329C064D7ACE9B04C51C05787927F1C3329C02347DF49B04C51C0B9C49107223329C0F4DCE79DAF4C51C054B76922223329C0184339D1AE4C51C042A89C40223329C048E17A14AE4C51C010DBCCD71B3329C007DB3E9AA04C51C0636996BA1A3329C0E9656F84A04C51C05ADD45F35B3329C04E9B711AA24C51C033373A42613329C06CE22E0CA24C51C066D24B31AC3329C05F454607A44C51C065170CAEB93329C07C8C03F9A34C51C0E475B3F5C23329C0"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-69.1991349 -12.6006222, -69.199136 -12.599842, -69.1982979 -12.5998268, -69.1982598 -12.599869, -69.1982188 -12.5998698, -69.19817 -12.5998707, -69.198125 -12.5998218, -69.1973024 -12.5998133, -69.1972972 -12.6003109, -69.197394 -12.6003514, -69.1973906 -12.6009231, -69.1975115 -12.601026, -69.1975081 -12.6010968)" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 3, "XMax": 7.0, "XMin": 1.0, "YMax": 8.0, "YMin": 2.0, "Binary": hex("010200000003000000000000000000F03F0000000000000040000000000000104000000000000014400000000000001C400000000000002040"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[1,2],[4,5],[7,8]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (1 2, 4 5, 7 8)" }
+{ "Type": "MultiPoint", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 4, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiPoint\",\"coordinates\":[[10,40],[40,30],[20,20],[30,10]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "MULTIPOINT ((10 40), (40 30), (20 20), (30 10))" }
+{ "Type": "MultiPolygon", "Area": 712.5, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 11, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 5.0, "Binary": hex("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440"), "GeoJSON": "{\"type\":\"MultiPolygon\",\"coordinates\":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],\"crs\":null}", "WKT": "MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)), ((20 35, 10 30, 10 10, 30 5, 45 20, 20 35), (30 20, 20 15, 20 25, 30 20)))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -71.1043443253471, "XMin": -71.1043443253471, "YMax": 42.3150676015829, "YMin": 42.3150676015829, "Binary": hex("0101000000E538D293ADC651C0F3699A2254284540"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-71.1043443253471,42.3150676015829],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-71.1043443253471 42.3150676015829)" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 3, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01E9030000000000000000F03F00000000000000400000000000000840"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT Z (1 2 3)" }
+{ "Type": "Polygon", "Area": 1.3755215000294761E-8, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 6, "XMax": 8.760178, "XMin": 8.7599721, "YMax": 49.7103478, "YMin": 49.7102133, "Binary": hex("01030000000100000007000000B1BE26101B852140ED20C033EBDA4840C11DA8531E8521407694394CEADA4840BAF8DB9E20852140D5F89683EADA484098EF2AB5288521409557F844E8DA48400247020D3685214041F74086E9DA484014F131B32A8521408DE43CADECDA4840B1BE26101B852140ED20C033EBDA4840"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))" }
+{ "Type": "MultiLineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 7, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiLineString\",\"coordinates\":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],\"crs\":null}", "WKT": "MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 4, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01B90B0000000000000000F03F000000000000004000000000000008400000000000001040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3,4],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT ZM (1 2 3 4)" }
+{ "Type": "Polygon", "Area": 928.625, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 743265.625, "XMin": 743238.0, "YMax": 2967450.0, "YMin": 2967416.0, "Binary": hex("01030000000100000005000000000000008CAE264100000000BCA3464100000040C3AE264100000000BCA3464100000000C2AE264100000000CDA34641000000008CAE264100000000CDA34641000000008CAE264100000000BCA34641"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416))" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 2, "XMax": -113.98, "XMin": -113.981, "YMax": 39.198, "YMin": 39.195, "Binary": hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-113.98,39.198],[-113.981,39.195]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-113.98 39.198, -113.981 39.195)" }
+{ "Type": "Polygon", "Area": 1.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 11.0, "XMin": 10.0, "YMax": 11.0, "YMin": 10.0, "Binary": hex("010300000001000000050000000000000000002440000000000000244000000000000026400000000000002440000000000000264000000000000026400000000000002440000000000000264000000000000024400000000000002440"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -118.4, "XMin": -118.4, "YMax": 33.93, "YMin": 33.93, "Binary": hex("01010000009A99999999995DC0D7A3703D0AF74040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-118.4,33.93],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-118.4 33.93)" }
+{ "Type": "Polygon", "Area": 675.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 7, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 10.0, "Binary": hex("0103000000020000000500000000000000008041400000000000002440000000000080464000000000008046400000000000002E40000000000000444000000000000024400000000000003440000000000080414000000000000024400400000000000000000034400000000000003E40000000000080414000000000008041400000000000003E40000000000000344000000000000034400000000000003E40"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
new file mode 100644
index 0000000..950f68f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
@@ -0,0 +1,4 @@
+{ "X": -71.1043443253471, "Y": 42.3150676015829, "Z": 0.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": 4.0 }
+{ "X": -118.4, "Y": 33.93, "Z": 0.0, "M": NaN }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
new file mode 100644
index 0000000..044e31a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
@@ -0,0 +1,4 @@
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[10,10],[11,10],[11,11],[10,11],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 1, "ExteriorRing": {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
new file mode 100644
index 0000000..7967722
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
@@ -0,0 +1,4 @@
+{ "Length": 0.004058119099397876, "Boundary": {"type":"MultiPoint","coordinates":[[-69.1991349,-12.6006222],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 78.9292222699217, "Boundary": {"type":"MultiPoint","coordinates":[[10,10],[10,40],[40,40],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 0.0031622776601655037, "Boundary": {"type":"MultiPoint","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 8.48528137423857, "Boundary": {"type":"MultiPoint","coordinates":[[1,2],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
new file mode 100644
index 0000000..a1e16dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
@@ -0,0 +1 @@
+{"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
new file mode 100644
index 0000000..f884259
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
@@ -0,0 +1,10 @@
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 1.4142135623730951, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": true, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 2.8284271247461903, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": true, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": true, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
new file mode 100644
index 0000000..d9b7206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
@@ -0,0 +1,10 @@
+{ "Union": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,0],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[-1,-1],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,3]],[[3,3],[4,2],[5,2]],[[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"LineString","coordinates":[[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[0,2],[1,2],[2,2],[1,1]],[[5,2],[4,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"Point","coordinates":[2,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Polygon","coordinates":[[[0.25,0.25],[0.5,0.25],[0.5,0.5],[0.25,0.5],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]],[[0.25,0.25],[0.25,0.5],[0.5,0.5],[0.5,0.25],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
new file mode 100644
index 0000000..9c4866f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$15(ASC), $$16(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$15, $$16, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$21) (ASC, $$22)
+                  -- STABLE_SORT [$$21(ASC), $$22(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$21, $$22])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$19, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$19] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
new file mode 100644
index 0000000..3d466ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
+{ "l_orderkey": 33, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5, "l_extendedprice": 5190.65, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". stealthily bold exc" }
+{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
+{ "l_orderkey": 35, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7, "l_extendedprice": 7147.84, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-01-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the carefully regular " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
new file mode 100644
index 0000000..29357ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC), $$13(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$12, $$13, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$17, $$18, 2, $$17, $$18, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$17) (ASC, $$18)
+                  -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$17, $$18])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$16, $$17, $$18] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$15, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$15] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
new file mode 100644
index 0000000..eeedd56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
new file mode 100644
index 0000000..b60a0ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
new file mode 100644
index 0000000..afc422a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
@@ -0,0 +1,5 @@
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects  Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
new file mode 100644
index 0000000..daeeeb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$10(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$10, $$paper] <- test.DBLP1 limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
new file mode 100644
index 0000000..e9f7751
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
@@ -0,0 +1,5 @@
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better  A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story  Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
new file mode 100644
index 0000000..91a2dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
@@ -0,0 +1 @@
+.*"processedObjects":1.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
new file mode 100644
index 0000000..8b095a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
@@ -0,0 +1 @@
+{ "v1": true, "v2": true, "v3": [ true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
new file mode 100644
index 0000000..4be3e9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
@@ -0,0 +1 @@
+{ "t1": 6, "t2": 6, "t3": 6, "t4": 6, "t5": [ true, true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
index 920f7b4..cf13212 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
@@ -1 +1 @@
-{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false }
+{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false, "25": false, "26": false, "27": true, "28": true, "29": true, "30": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
new file mode 100644
index 0000000..abab248
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
@@ -0,0 +1 @@
+{ "e": 2.71, "pi": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
new file mode 100644
index 0000000..e1310b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
@@ -0,0 +1 @@
+{ "t1": 180 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
index 47c4e3c..775745b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
@@ -1 +1 @@
-{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
+{ "result1": 2.0, "result2": 1.0, "result3": 0.6666666666666666, "result4": -0.5, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
new file mode 100644
index 0000000..c3d22c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
@@ -0,0 +1 @@
+{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
index 83c2d70..890417b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
@@ -1 +1 @@
-{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
+{ "result1": 3.0, "result2": 1.5, "result3": 1.0, "result4": -0.75, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
new file mode 100644
index 0000000..788a291
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
@@ -0,0 +1 @@
+{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
index a6fea08..582b44d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
@@ -1 +1 @@
-{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
+{ "result1": -4.0, "result2": -2.0, "result3": -1.3333333333333333, "result4": 1.0, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
new file mode 100644
index 0000000..af75a4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
@@ -0,0 +1 @@
+{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
index 1def226..591ed6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
@@ -1 +1 @@
-{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
+{ "result1": 1.0, "result2": 0.5, "result3": 0.3333333333333333, "result4": -0.25, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
new file mode 100644
index 0000000..f232ecf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
@@ -0,0 +1 @@
+{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
new file mode 100644
index 0000000..6091a8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
@@ -0,0 +1 @@
+{ "t1": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
new file mode 100644
index 0000000..61d0506
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true ], "t2": { "b": 2 }, "t3": { "a": 1 }, "t4": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 18, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 } ], "t6": [ { "tweetid": "1", "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, { "tweetid": "10", "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, { "tweetid": "11", "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, { "tweetid": "12", "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, { "tweetid": "2", "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, { "tweetid": "3", "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, { "tweetid": "4", "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, { "tweetid": "5", "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, { "tweetid": "6", "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, { "tweetid": "7", "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, { "tweetid": "8", "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, { "tweetid": "9", "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
new file mode 100644
index 0000000..197fec1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true ], "t2": { "a": 1, "c": 2 }, "t3": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "language": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t6": [ { "tweetid": "1", "user_details": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
new file mode 100644
index 0000000..87269dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true, true ], "t2": "1", "t3": { "b": "1" }, "t4": [ 1, 2 ], "t5": [ { "$1": "en" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
new file mode 100644
index 0000000..a2b8b2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true ], "t2": "", "t3": "dcba", "t4": [ 98, 8457, 247, 8451, 215, 97 ], "t5": [ "ba", "cba", "dcba" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
index ac9dedd..326e22f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
@@ -1 +1 @@
-{ "str2": "ld", "str4": "g", "str6": "", "str8": "This is a test string", "str10": "This is a test string", "str13": "gThis is a another test string", "str14": "Irvine" }
+{ "str2": "ld", "str4": "g", "str6": "", "str8": "This is a test string", "str10": "string", "str13": "gThis is a another test string", "str14": "Irvine" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
index 5b0c7b9..197a7af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
@@ -1 +1 @@
-{ "result1": "HEllow" }
+{ "result1": "low" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
index b92a705..a8e64f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
@@ -1 +1 @@
-[ "ab", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
+[ "g", "ab", "ab", "bc", "cd" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
index e341861..a11b25b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
@@ -1 +1 @@
-[ "abcdefg", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
+[ "g", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm
new file mode 100644
index 0000000..4dbe13c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm
@@ -0,0 +1 @@
+"broker3"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm
new file mode 100644
index 0000000..bf0d87a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm
@@ -0,0 +1 @@
+4
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
index c76f17a..d99fd3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
@@ -1,2 +1 @@
 -0.1
--0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
new file mode 100644
index 0000000..13a1d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
@@ -0,0 +1 @@
+-0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
new file mode 100644
index 0000000..90fd039
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
new file mode 100644
index 0000000..05bd4a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
new file mode 100644
index 0000000..5e1007a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
new file mode 100644
index 0000000..6442d38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
index a86d0a4..9d1c274 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
@@ -37,7 +37,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      - LiteralExpr [LONG] [1]
+      - LiteralExpr [LONG] [6]
     ]
   )
   (
@@ -63,7 +63,7 @@
           FunctionCall null.string-length@1[
             LiteralExpr [STRING] [UC Irvine]
           ]
-          /
+          div
           LiteralExpr [LONG] [2]
         ]
         -
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
index 414ad6e..8ff93d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
@@ -6,7 +6,7 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      - LiteralExpr [LONG] [1]
+      - LiteralExpr [LONG] [3]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 5c6b100..85779fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1668,7 +1668,7 @@
     <test-case FilePath="dml">
       <compilation-unit name="insert-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>Duplicate field name &quot;id&quot;</expected-error>
+        <expected-error>ASX1006: Duplicate field name &quot;id&quot;</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1689,13 +1689,13 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_03">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
-        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1716,7 +1716,7 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-no-field">
         <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
-        <expected-error>ASX1014: Field "not_id" is not found [CompilationException]</expected-error>
+        <expected-error>ASX1014: Field "not_id" is not found</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -4477,7 +4477,7 @@
     <test-case FilePath="numeric">
       <compilation-unit name="query-issue355">
         <output-dir compare="Text">query-issue355</output-dir>
-        <expected-error>java.lang.NumberFormatException: For input string: "10000000000000000000"</expected-error>
+        <expected-error>For input string: "10000000000000000000"</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -4923,7 +4923,7 @@
         <output-dir compare="Text">alltypes_01</output-dir>
       </compilation-unit>
     </test-case>
-    <test-case FilePath="scan">
+    <!--test-case FilePath="scan">
       <compilation-unit name="alltypes_01">
         <parameter name="wrapper-array" value="true" />
         <output-dir compare="Text">alltypes_01-wrapped</output-dir>
@@ -4944,7 +4944,7 @@
       <compilation-unit name="alltypes_01">
         <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
       </compilation-unit>
-    </test-case>
+    </test-case-->
     <test-case FilePath="scan">
       <compilation-unit name="alltypes_02">
         <output-dir compare="Text">alltypes_02</output-dir>
@@ -6525,19 +6525,19 @@
     <test-case FilePath="load">
       <compilation-unit name="csv_05"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_05</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_06"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_06</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_07"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_07</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -6575,7 +6575,7 @@
     <test-case FilePath="load">
       <compilation-unit name="issue650_query">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Unable to load dataset Users since it does not exist</expected-error>
+        <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 2ee294a..4376ca6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -19,8 +19,9 @@
 <!DOCTYPE test-suite [
   <!ENTITY ObjectsQueries SYSTEM "queries_sqlpp/objects/ObjectsQueries.xml">
   <!ENTITY AsyncDeferredQueries SYSTEM "queries_sqlpp/async-deferred/AsyncDeferredQueries.xml">
+  <!ENTITY GeoQueries SYSTEM "queries_sqlpp/geojson/GeoJSONQueries.xml">
 ]>
-<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
   &AsyncDeferredQueries;
   <test-group name="flwor">
     <test-case FilePath="flwor">
@@ -1005,6 +1006,11 @@
   </test-group>
   <test-group name="comparison">
     <test-case FilePath="comparison">
+      <compilation-unit name="secondary_idx_lookup">
+        <output-dir compare="Text">secondary_idx_lookup</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
       <compilation-unit name="year_month_duration_order">
         <output-dir compare="Text">year_month_duration_order</output-dir>
       </compilation-unit>
@@ -1263,6 +1269,32 @@
         <output-dir compare="Text">least_mixed</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="missingif">
+        <output-dir compare="Text">missingif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="nullif">
+        <output-dir compare="Text">nullif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="nanif">
+        <output-dir compare="Text">nanif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="posinfif">
+        <output-dir compare="Text">posinfif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="neginfif">
+        <output-dir compare="Text">neginfif</output-dir>
+      </compilation-unit>
+    </test-case>
+
   </test-group>
   <test-group name="constructor">
     <test-case FilePath="constructor">
@@ -1826,6 +1858,7 @@
       <compilation-unit name="insert-duplicated-keys">
         <output-dir compare="Text">insert-duplicated-keys</output-dir>
         <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1874,13 +1907,13 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>Parse error at (0, 5): ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_03">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
-        <expected-error>Parse error at (0, 5): ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1920,6 +1953,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname-qualified">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
       <compilation-unit name="insert-returning-fieldname-implicit">
         <output-dir compare="Text">insert-returning-fieldname</output-dir>
         <expected-error>Need an alias for the enclosed expression</expected-error>
@@ -3604,6 +3642,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="case_08">
+        <output-dir compare="Text">case_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="dataset_nodegroup">
         <output-dir compare="Text">dataset_nodegroup</output-dir>
       </compilation-unit>
@@ -3612,7 +3655,7 @@
       <compilation-unit name="partition-by-nonexistent-field">
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
         <expected-error>Field "id" is not found</expected-error>
-        <expected-error>Cannot find dataset testds in dataverse test</expected-error>
+        <expected-error>Cannot find dataset with name testds in dataverse test</expected-error>
         <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds!</expected-error>
       </compilation-unit>
     </test-case>
@@ -3657,6 +3700,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="random">
+        <output-dir compare="Text">random</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="stable_sort">
         <output-dir compare="Text">stable_sort</output-dir>
       </compilation-unit>
@@ -4420,6 +4468,11 @@
           <output-dir compare="Text">rtree-secondary-index-open</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="rtree-secondary-index-optional">
+          <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+        </compilation-unit>
+      </test-case>
     </test-group>
     <test-group name="nested-index/external-indexing">
       <test-case FilePath="nested-index/external-indexing">
@@ -4602,6 +4655,12 @@
         <output-dir compare="Text">nestrecord</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="nestrecords">
+      <compilation-unit name="nested-optional-pk">
+        <output-dir compare="Text">nested-optional-pk</output-dir>
+        <expected-error>ASX1021: The primary key field "nested.id" cannot be nullable</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="null-missing">
     <test-case FilePath="null-missing">
@@ -4802,6 +4861,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="numeric">
+      <compilation-unit name="const">
+        <output-dir compare="Text">const</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="degrees">
+        <output-dir compare="Text">degrees</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
       <compilation-unit name="divide_double">
         <output-dir compare="Text">divide_double</output-dir>
       </compilation-unit>
@@ -4912,6 +4981,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="numeric">
+      <compilation-unit name="radians">
+        <output-dir compare="Text">radians</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
       <compilation-unit name="round-half-to-even0">
         <output-dir compare="Text">round-half-to-even0</output-dir>
       </compilation-unit>
@@ -5363,6 +5437,7 @@
       <compilation-unit name="query-issue410">
         <output-dir compare="Text">query-issue410</output-dir>
         <expected-error>Field type double can't be promoted to type string</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="open-closed">
@@ -6238,6 +6313,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="string">
+      <compilation-unit name="reverse">
+        <output-dir compare="Text">reverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
       <compilation-unit name="rtrim">
         <output-dir compare="Text">rtrim</output-dir>
       </compilation-unit>
@@ -8074,36 +8154,42 @@
       <compilation-unit name="drop-dependency-1">
         <output-dir compare="Text">drop-dependency-1</output-dir>
         <expected-error>Cannot drop dataverse. Function B.f0@2 depends on function C.f1@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-2">
         <output-dir compare="Text">drop-dependency-2</output-dir>
         <expected-error>Cannot drop dataverse. Function B.f2@2 depends on dataset C.TweetMessages</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-3">
         <output-dir compare="Text">drop-dependency-3</output-dir>
         <expected-error>Cannot drop function C.f1@2 being used by function B.f0@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-4">
         <output-dir compare="Text">drop-dependency-4</output-dir>
         <expected-error>Cannot drop dataset C.TweetMessages being used by function B.f2@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-5">
         <output-dir compare="Text">drop-dependency-5</output-dir>
         <expected-error>Cannot drop function C.f1@2 being used by function C.f0@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-6">
         <output-dir compare="Text">drop-dependency-6</output-dir>
         <expected-error>Cannot drop dataset C.TweetMessages being used by function C.f2@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
@@ -8348,18 +8434,21 @@
       <compilation-unit name="csv_05">
         <output-dir compare="Text">csv_05</output-dir>
         <expected-error>At record: 1, field#: 4 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_06">
         <output-dir compare="Text">csv_06</output-dir>
         <expected-error>At record: 1, field#: 3 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_07">
         <output-dir compare="Text">csv_07</output-dir>
         <expected-error>At record: 1, field#: 3 -  A quote enclosing a field needs to be followed by the delimiter</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8381,12 +8470,14 @@
       <compilation-unit name="issue14_query">
         <output-dir compare="Text">issue14_query</output-dir>
         <expected-error>Unspecified parameter: format</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="issue315_query">
         <output-dir compare="Text">none</output-dir>
         <expected-error>Invalid path</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8397,7 +8488,7 @@
     <test-case FilePath="load">
       <compilation-unit name="issue650_query">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Unable to load dataset Users since it does not exist</expected-error>
+        <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8430,6 +8521,7 @@
       <compilation-unit name="duplicate-key-error">
         <output-dir compare="Text">none</output-dir>
         <expected-error>Loading duplicate keys into the primary storage</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8460,6 +8552,13 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="function">
+    <test-case FilePath="function">
+      <compilation-unit name="issue-2394">
+        <output-dir compare="Text">issue-2394</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="feeds">
     <test-case FilePath="feeds">
       <compilation-unit name="feeds_07">
@@ -8592,6 +8691,7 @@
         <output-dir compare="Text">twitter-feed</output-dir>
         <expected-error>Twitter4J library not found!</expected-error>
         <expected-error>Unknown source feed: TwitterFeed</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8599,6 +8699,7 @@
         <output-dir compare="Text">revised-tweet-parser</output-dir>
         <expected-error>Twitter4J library not found!</expected-error>
         <expected-error>Unknown source feed: TwitterFeed</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8645,12 +8746,14 @@
       <compilation-unit name="start-started-feed">
         <output-dir compare="Text">start-started-feed</output-dir>
         <expected-error>experiments.TweetFeed(Feed) is already started</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
       <compilation-unit name="stop-stopped-feed">
         <output-dir compare="Text">stop-stopped-feed</output-dir>
         <expected-error>new_experiments.TweetFeed(Feed) cannot be stopped because its state is STOPPED</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -9330,6 +9433,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="filters">
+      <compilation-unit name="filter-auto-key">
+        <output-dir compare="Text">filter-auto-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
       <compilation-unit name="load">
         <output-dir compare="Text">load</output-dir>
       </compilation-unit>
@@ -9640,6 +9748,11 @@
         <output-dir compare="Text">multiple-secondaries</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="upsert-case-returning">
+        <output-dir compare="Text">upsert-case-returning</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="json-parser">
     <test-case FilePath="json-parser">
@@ -9651,6 +9764,7 @@
       <compilation-unit name="numeric-tinyint-overflow">
         <output-dir compare="Text">numeric-tinyint</output-dir>
         <expected-error>Numeric value (1000) out of range of Java byte</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9672,12 +9786,14 @@
       <compilation-unit name="duplicate-fields">
         <output-dir compare="Text">duplicate-fields</output-dir>
         <expected-error>Duplicate field 'field'</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="malformed-json">
         <output-dir compare="Text">malformed-json</output-dir>
         <expected-error>Unexpected character ('}' (code 125)): was expecting double-quote to start field name</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9689,12 +9805,14 @@
       <compilation-unit name="nonoptional-missing">
         <output-dir compare="Text">nonoptional-missing</output-dir>
         <expected-error>ASX3075: Closed field missing_value has null value</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="nonoptional-null">
         <output-dir compare="Text">nonoptional-null</output-dir>
         <expected-error>ASX3075: Closed field null_value has null value</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9706,18 +9824,21 @@
       <compilation-unit name="spatial-line-3-points">
         <output-dir compare="Text">spatial-line-3-points</output-dir>
         <expected-error>Line must have 4 coordinates</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="spatial-polygon-unclosed">
         <output-dir compare="Text">spatial-polygon-unclosed</output-dir>
         <expected-error>Unclosed polygon is not supported</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="spatial-polygon-with-hole">
         <output-dir compare="Text">spatial-polygon-with-hole</output-dir>
         <expected-error>Only simple geometries are supported (Point, LineString and Polygon without holes)</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9729,48 +9850,56 @@
       <compilation-unit name="type-mismatch">
         <output-dir compare="Text">type-mismatch</output-dir>
         <expected-error>ASX3054: Mismatch Type, expecting a value of type string</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-circle">
         <output-dir compare="Text">unsupported-type-circle</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type circle</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-daytimeduration">
         <output-dir compare="Text">unsupported-type-daytimeduration</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type daytimeduration</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-duration">
         <output-dir compare="Text">unsupported-type-duration</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type duration</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-interval">
         <output-dir compare="Text">unsupported-type-interval</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type interval</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-multiset">
         <output-dir compare="Text">unsupported-type-multiset</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type multiset</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-point3d">
         <output-dir compare="Text">unsupported-type-point3d</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type point3d</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-rectangle">
         <output-dir compare="Text">unsupported-type-rectangle</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type rectangle</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -9796,4 +9925,27 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="limit">
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-primary-scan">
+        <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
+      </compilation-unit>
+    </test-case>
+     <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-scan-select">
+         <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
+       </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-lookup">
+         <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+       </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-lookup-select">
+         <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+       </compilation-unit>
+    </test-case>
+  </test-group>
+  &GeoQueries;
 </test-suite>
diff --git a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
index fd6c657..7aa2cf9 100644
--- a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
+++ b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey, SUM(o_orderkey_count) AS c_count
         FROM  (
                 SELECT c.c_custkey,
-                       COLL_COUNT(
+                       STRICT_COUNT(
                            (
                             select element o.o_orderkey
                             from  Orders o
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
new file mode 100644
index 0000000..7075739
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.hyracks.api.application.INCServiceContext;
+
+@FunctionalInterface
+public interface IDatasetInfoProvider extends Serializable {
+    /**
+     * Get the dataset info
+     *
+     * @param serviceCtx
+     *            the nc service context
+     * @return the dataset info object
+     */
+    DatasetInfo getDatasetInfo(INCServiceContext serviceCtx);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
index 9551935..7e9879a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
@@ -40,7 +40,11 @@
         /**
          * Extends Language Syntax and Algebric Operations
          */
-        LANG
+        LANG,
+        /**
+         * Extends Recovery Capabilities
+         */
+        RECOVERY
     }
 
     /**
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
new file mode 100644
index 0000000..5fd37e9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.common.IResource;
+
+@FunctionalInterface
+public interface ILSMComponentIdGeneratorFactory extends Serializable {
+    /**
+     * Get the Id generator for the resource
+     *
+     * @param serviceCtx
+     * @param resource
+     * @return
+     * @throws HyracksDataException
+     */
+    ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
+            throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 9eb6259..b6581ec2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.IReplicaManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.INCServiceContext;
@@ -65,7 +66,8 @@
 
     IResourceIdFactory getResourceIdFactory();
 
-    void initialize(boolean initialRun) throws IOException, AlgebricksException;
+    void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun)
+            throws IOException, AlgebricksException;
 
     void setShuttingdown(boolean b);
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
index c15f8a7..4ee00cd 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
@@ -38,7 +38,7 @@
         TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of pages in the transaction log tail"),
         TXN_LOG_BUFFER_PAGESIZE(
                 INTEGER_BYTE_UNIT,
-                StorageUtil.getIntSizeInBytes(128, KILOBYTE),
+                StorageUtil.getIntSizeInBytes(4, MEGABYTE),
                 "The page size (in bytes) for transaction log buffer"),
         TXN_LOG_PARTITIONSIZE(
                 LONG_BYTE_UNIT,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index b7af0b6..3ff13cb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -37,8 +37,7 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.declareActiveIOOperation();
         }
     }
@@ -47,14 +46,14 @@
     public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
+            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
     @Override
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.undeclareActiveIOOperation();
         }
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
index 41461ec..a14fac8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
@@ -125,7 +125,7 @@
                 }
             }
             ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(lsmIndex.getIOOperationCallback(), mergableComponents);
+            accessor.scheduleMerge(mergableComponents);
         }
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index f4d764a..6e2e320 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -55,7 +55,7 @@
         this.setRegistered(false);
         this.setMemoryAllocated(false);
         this.logManager = logManager;
-        waitLog.setLogType(LogType.WAIT);
+        waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
         waitLog.computeAndSetLogSize();
     }
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
new file mode 100644
index 0000000..597b5da
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.context;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public class DatasetInfoProvider implements IDatasetInfoProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final int datasetId;
+
+    public DatasetInfoProvider(int datasetId) {
+        this.datasetId = datasetId;
+    }
+
+    @Override
+    public DatasetInfo getDatasetInfo(INCServiceContext serviceCtx) {
+        IDatasetLifecycleManager dslcManager =
+                ((INcApplicationContext) serviceCtx.getApplicationContext()).getDatasetLifecycleManager();
+        return dslcManager.getDatasetInfo(datasetId);
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
index 83e3144..85eacf8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.common.context;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.common.IResource;
 
 /**
@@ -51,5 +51,4 @@
         int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
         return dslcManager.getComponentIdGenerator(datasetId, partition);
     }
-
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index b715eec..50a4bef 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -23,7 +23,6 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -33,26 +32,24 @@
 import org.apache.asterix.common.api.IDatasetMemoryManager;
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
@@ -68,7 +65,7 @@
     private final ILocalResourceRepository resourceRepository;
     private final IDatasetMemoryManager memoryManager;
     private final ILogManager logManager;
-    private final LogRecord logRecord;
+    private final LogRecord waitLog;
     private final int numPartitions;
     private volatile boolean stopped = false;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
@@ -82,7 +79,9 @@
         this.memoryManager = memoryManager;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
         this.numPartitions = numPartitions;
-        logRecord = new LogRecord();
+        waitLog = new LogRecord();
+        waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
+        waitLog.computeAndSetLogSize();
     }
 
     @Override
@@ -348,7 +347,7 @@
     }
 
     private void populateOpTrackerAndIdGenerator(DatasetResource dataset, int partition) {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(storageProperties.getMemoryComponentsNum());
         PrimaryIndexOperationTracker opTracker = new PrimaryIndexOperationTracker(dataset.getDatasetID(), partition,
                 logManager, dataset.getDatasetInfo(), idGenerator);
         dataset.setPrimaryIndexOperationTracker(partition, opTracker);
@@ -369,7 +368,9 @@
     @Override
     public synchronized void flushAllDatasets() throws HyracksDataException {
         for (DatasetResource dsr : datasets.values()) {
-            flushDatasetOpenIndexes(dsr, false);
+            if (dsr.getDatasetInfo().isOpen()) {
+                flushDatasetOpenIndexes(dsr, false);
+            }
         }
     }
 
@@ -398,11 +399,10 @@
             PrimaryIndexOperationTracker opTracker, long targetLSN) throws HyracksDataException {
         int partition = opTracker.getPartition();
         for (ILSMIndex lsmIndex : dsr.getDatasetInfo().getDatasetPartitionOpenIndexes(partition)) {
-            AbstractLSMIOOperationCallback ioCallback =
-                    (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+            LSMIOOperationCallback ioCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
             if (!(lsmIndex.isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()
                     || opTracker.isFlushLogCreated() || opTracker.isFlushOnExit())) {
-                long firstLSN = ioCallback.getFirstLSN();
+                long firstLSN = ioCallback.getPersistenceLsn();
                 if (firstLSN < targetLSN) {
                     LOGGER.info("Checkpoint flush dataset {} partition {}", dsr.getDatasetID(), partition);
                     opTracker.setFlushOnExit(true);
@@ -422,90 +422,48 @@
      */
     private void flushDatasetOpenIndexes(DatasetResource dsr, boolean asyncFlush) throws HyracksDataException {
         DatasetInfo dsInfo = dsr.getDatasetInfo();
+        if (!dsInfo.isOpen()) {
+            throw new IllegalStateException("flushDatasetOpenIndexes is called on a dataset that is closed");
+        }
         if (dsInfo.isExternal()) {
             // no memory components for external dataset
             return;
         }
+        // ensure all in-flight flushes gets scheduled
+        logManager.log(waitLog);
         for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
             // flush each partition one by one
             if (primaryOpTracker.getNumActiveOperations() > 0) {
                 throw new IllegalStateException(
                         "flushDatasetOpenIndexes is called on a dataset with currently active operations");
             }
-            int partition = primaryOpTracker.getPartition();
-            Collection<ILSMIndex> indexes = dsInfo.getDatasetPartitionOpenIndexes(partition);
-            ILSMIndex flushIndex = null;
-            for (ILSMIndex lsmIndex : indexes) {
-                if (!lsmIndex.isCurrentMutableComponentEmpty()) {
-                    flushIndex = lsmIndex;
-                    break;
-                }
-            }
-            if (flushIndex == null) {
-                // all open indexes are empty, nothing to flush
-                continue;
-            }
-            LSMComponentId componentId = (LSMComponentId) flushIndex.getCurrentMemoryComponent().getId();
-            ILSMComponentIdGenerator idGenerator = getComponentIdGenerator(dsInfo.getDatasetID(), partition);
-            idGenerator.refresh();
-
-            if (dsInfo.isDurable()) {
-
-                synchronized (logRecord) {
-                    TransactionUtil.formFlushLogRecord(logRecord, dsInfo.getDatasetID(), partition,
-                            componentId.getMinId(), componentId.getMaxId(), null);
-                    try {
-                        logManager.log(logRecord);
-                    } catch (ACIDException e) {
-                        throw new HyracksDataException("could not write flush log while closing dataset", e);
-                    }
-
-                    try {
-                        //notification will come from LogBuffer class (notifyFlushTerminator)
-                        logRecord.wait();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                        throw HyracksDataException.create(e);
-                    }
-                }
-            }
-            for (ILSMIndex index : indexes) {
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
-            }
-
-            if (asyncFlush) {
-                for (ILSMIndex index : indexes) {
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                }
-            } else {
-                for (ILSMIndex index : indexes) {
-                    // TODO: This is not efficient since we flush the indexes sequentially.
-                    // Think of a way to allow submitting the flush requests concurrently.
-                    // We don't do them concurrently because this may lead to a deadlock scenario
-                    // between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                    // Wait for the above flush op.
-                    dsInfo.waitForIO();
-                }
-            }
+            primaryOpTracker.setFlushOnExit(true);
+            primaryOpTracker.flushIfNeeded();
         }
-
+        // ensure requested flushes were scheduled
+        logManager.log(waitLog);
+        if (!asyncFlush) {
+            List<FlushOperation> flushes = new ArrayList<>();
+            for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
+                flushes.addAll(primaryOpTracker.getScheduledFlushes());
+            }
+            LSMIndexUtil.waitFor(flushes);
+        }
     }
 
     private void closeDataset(DatasetResource dsr) throws HyracksDataException {
         // First wait for any ongoing IO operations
         DatasetInfo dsInfo = dsr.getDatasetInfo();
-        dsInfo.waitForIO();
         try {
             flushDatasetOpenIndexes(dsr, false);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
+        // wait for merges that were scheduled due to the above flush
+        // ideally, we shouldn't need this since merges should still work.
+        // They don't need a special memory budget but there is a problem
+        // for some merge policies that need to access dataset info (correlated prefix)
+        dsInfo.waitForIO();
         for (IndexInfo iInfo : dsInfo.getIndexes().values()) {
             closeIndex(iInfo);
         }
@@ -517,7 +475,9 @@
     public synchronized void closeAllDatasets() throws HyracksDataException {
         ArrayList<DatasetResource> openDatasets = new ArrayList<>(datasets.values());
         for (DatasetResource dsr : openDatasets) {
-            closeDataset(dsr);
+            if (dsr.isOpen()) {
+                closeDataset(dsr);
+            }
         }
     }
 
@@ -624,7 +584,7 @@
     @Override
     public void flushDataset(IReplicationStrategy replicationStrategy) throws HyracksDataException {
         for (DatasetResource dsr : datasets.values()) {
-            if (replicationStrategy.isMatch(dsr.getDatasetID())) {
+            if (dsr.isOpen() && replicationStrategy.isMatch(dsr.getDatasetID())) {
                 flushDatasetOpenIndexes(dsr, false);
             }
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 74848d5..a1d31d5 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -19,11 +19,16 @@
 
 package org.apache.asterix.common.context;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
@@ -33,18 +38,24 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
-public class PrimaryIndexOperationTracker extends BaseOperationTracker {
-
+public class PrimaryIndexOperationTracker extends BaseOperationTracker implements IoOperationCompleteListener {
+    private static final Logger LOGGER = LogManager.getLogger();
     private final int partition;
     // Number of active operations on an ILSMIndex instance.
     private final AtomicInteger numActiveOperations;
@@ -52,6 +63,7 @@
     private final ILSMComponentIdGenerator idGenerator;
     private boolean flushOnExit = false;
     private boolean flushLogCreated = false;
+    private final Map<String, FlushOperation> scheduledFlushes = new HashMap<>();
 
     public PrimaryIndexOperationTracker(int datasetID, int partition, ILogManager logManager, DatasetInfo dsInfo,
             ILSMComponentIdGenerator idGenerator) {
@@ -65,20 +77,9 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        super.beforeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             incrementNumActiveOperations(modificationCallback);
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.declareActiveIOOperation();
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
-        if (opType == LSMOperationType.REPLICATE) {
-            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
@@ -86,12 +87,10 @@
     public synchronized void completeOperation(ILSMIndex index, LSMOperationType opType,
             ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
             throws HyracksDataException {
+        super.completeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             decrementNumActiveOperations(modificationCallback);
             flushIfNeeded();
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
         }
     }
 
@@ -117,16 +116,46 @@
             }
         }
 
+        ILSMIndex primaryLsmIndex = null;
         if (needsFlush || flushOnExit) {
+            flushOnExit = false;
             // make the current mutable components READABLE_UNWRITABLE to stop coming modify operations from entering
             // them until the current flush is scheduled.
             LSMComponentId primaryId = null;
+            //Double check that the primary index has been modified
+
+            synchronized (this) {
+                if (numActiveOperations.get() > 0) {
+                    throw new IllegalStateException(
+                            "Can't request a flush on an index with active operations: " + numActiveOperations.get());
+                }
+                for (ILSMIndex lsmIndex : indexes) {
+                    if (lsmIndex.isPrimaryIndex()) {
+                        if (lsmIndex.isCurrentMutableComponentEmpty()) {
+                            LOGGER.info("Primary index on dataset {} and partition {} is empty... skipping flush",
+                                    dsInfo.getDatasetID(), partition);
+                            return;
+                        }
+                        primaryLsmIndex = lsmIndex;
+                        break;
+                    }
+                }
+            }
+            if (primaryLsmIndex == null) {
+                throw new IllegalStateException(
+                        "Primary index not found in dataset " + dsInfo.getDatasetID() + " and partition " + partition);
+            }
             for (ILSMIndex lsmIndex : indexes) {
                 ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
                 synchronized (opTracker) {
                     ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+                    if (memComponent.getWriterCount() > 0) {
+                        throw new IllegalStateException(
+                                "Can't request a flush on a component with writers inside: Index:" + lsmIndex
+                                        + " Component:" + memComponent);
+                    }
                     if (memComponent.getState() == ComponentState.READABLE_WRITABLE && memComponent.isModified()) {
-                        memComponent.setState(ComponentState.READABLE_UNWRITABLE);
+                        memComponent.setUnwritable();
                     }
                     if (lsmIndex.isPrimaryIndex()) {
                         primaryId = (LSMComponentId) memComponent.getId();
@@ -134,10 +163,10 @@
                 }
             }
             if (primaryId == null) {
-                throw new IllegalStateException("Primary index not found in dataset " + dsInfo.getDatasetID());
+                throw new IllegalStateException("Primary index found in dataset " + dsInfo.getDatasetID()
+                        + " and partition " + partition + " and is modified but its component id is null");
             }
             LogRecord logRecord = new LogRecord();
-            flushOnExit = false;
             if (dsInfo.isDurable()) {
                 /*
                  * Generate a FLUSH log.
@@ -171,21 +200,41 @@
                 throw new IllegalStateException("Operation started while index was pending scheduling a flush");
             }
             idGenerator.refresh();
-            for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
-                //get resource
-                ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
-                //schedule flush after update
-                accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+            long flushLsn = logRecord.getLSN();
+            ILSMComponentId nextComponentId = idGenerator.getId();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+            synchronized (scheduledFlushes) {
+                for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
+                    ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+                    accessor.getOpContext().setParameters(flushMap);
+                    ILSMIOOperation flush = accessor.scheduleFlush();
+                    scheduledFlushes.put(flush.getTarget().getRelativePath(), (FlushOperation) flush);
+                    flush.addCompleteListener(this);
+                }
             }
         } finally {
             flushLogCreated = false;
         }
     }
 
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        synchronized (scheduledFlushes) {
+            scheduledFlushes.remove(operation.getTarget().getRelativePath());
+        }
+    }
+
+    public List<FlushOperation> getScheduledFlushes() {
+        synchronized (scheduledFlushes) {
+            Collection<FlushOperation> scheduled = scheduledFlushes.values();
+            List<FlushOperation> flushes = new ArrayList<FlushOperation>(scheduled.size());
+            flushes.addAll(scheduled);
+            return flushes;
+        }
+    }
+
     public int getNumActiveOperations() {
         return numActiveOperations.get();
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
index e844192..71d16f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
@@ -18,15 +18,19 @@
  */
 package org.apache.asterix.common.dataflow;
 
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import java.util.List;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 
 public class LSMIndexUtil {
 
+    private LSMIndexUtil() {
+    }
+
     public static void checkAndSetFirstLSN(AbstractLSMIndex lsmIndex, ILogManager logManager)
             throws HyracksDataException {
         // If the index has an empty memory component, we need to set its first LSN (For soft checkpoint)
@@ -34,11 +38,21 @@
             //prevent transactions from incorrectly setting the first LSN on a modified component by checking the index is still empty
             synchronized (lsmIndex.getOperationTracker()) {
                 if (lsmIndex.isCurrentMutableComponentEmpty()) {
-                    AbstractLSMIOOperationCallback ioOpCallback =
-                            (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                    ioOpCallback.setFirstLSN(logManager.getAppendLSN());
+                    LSMIOOperationCallback ioOpCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+                    ioOpCallback.setFirstLsnForCurrentMemoryComponent(logManager.getAppendLSN());
                 }
             }
         }
     }
+
+    public static void waitFor(List<? extends ILSMIOOperation> ioOperations) throws HyracksDataException {
+        for (int i = 0; i < ioOperations.size(); i++) {
+            try {
+                ioOperations.get(i).sync();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
index b3936de..1401c0a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -48,6 +49,7 @@
 
     public static final String KEY_INDEX = "Index";
     private final boolean isPrimary;
+    private final SourceLocation sourceLoc;
     // This class has both lsmIndex and index (in super class) pointing to the same object
     private AbstractLSMIndex lsmIndex;
     private int i = 0;
@@ -69,10 +71,11 @@
     public LSMInsertDeleteOperatorNodePushable(IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
             RecordDescriptor inputRecDesc, IndexOperation op, boolean isPrimary,
             IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
-            ITupleFilterFactory tupleFilterFactory) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, SourceLocation sourceLoc) throws HyracksDataException {
         super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, op, modCallbackFactory,
                 tupleFilterFactory);
         this.isPrimary = isPrimary;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -138,8 +141,8 @@
                         }
                         break;
                     default: {
-                        throw HyracksDataException.create(ErrorCode.INVALID_OPERATOR_OPERATION, op.toString(),
-                                LSMInsertDeleteOperatorNodePushable.class.getSimpleName());
+                        throw HyracksDataException.create(ErrorCode.INVALID_OPERATOR_OPERATION, sourceLoc,
+                                op.toString(), LSMInsertDeleteOperatorNodePushable.class.getSimpleName());
                     }
                 }
             }
@@ -147,10 +150,10 @@
             if (e.getErrorCode() == ErrorCode.INVALID_OPERATOR_OPERATION) {
                 throw e;
             } else {
-                throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, i);
+                throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, i);
             }
         } catch (Exception e) {
-            throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, i);
+            throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, i);
         }
 
         writeBuffer.ensureFrameSize(buffer.capacity());
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
index 6d58f6d..45661e4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
@@ -48,7 +48,7 @@
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
         return new LSMInsertDeleteOperatorNodePushable(ctx, partition, fieldPermutation, inputRecDesc, op, isPrimary,
-                indexHelperFactory, modCallbackFactory, tupleFilterFactory);
+                indexHelperFactory, modCallbackFactory, tupleFilterFactory, sourceLoc);
     }
 
     public boolean isPrimary() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
index dc884f1..a834a06 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
@@ -21,6 +21,7 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AsterixException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
@@ -33,9 +34,12 @@
         super(message);
     }
 
+    public AsterixException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+    }
+
     public AsterixException(int errorCode, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
-
     }
 
     /**
@@ -47,6 +51,11 @@
         super(cause);
     }
 
+    public AsterixException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+        addSuppressed(cause);
+    }
+
     public AsterixException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
         addSuppressed(cause);
@@ -60,6 +69,10 @@
         super(message, cause);
     }
 
+    public static AsterixException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        return new AsterixException(errorCode, sourceLoc, params);
+    }
+
     public static AsterixException create(int errorCode, Serializable... params) {
         return new AsterixException(errorCode, params);
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
index 693b1c0..0de6b72 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
@@ -22,14 +22,23 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CompilationException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
 
+    public CompilationException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+    }
+
     public CompilationException(int errorCode, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
     }
 
+    public CompilationException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+    }
+
     public CompilationException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index bd84832..de399eb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -76,7 +76,7 @@
     public static final int REJECT_NODE_UNREGISTERED = 33;
     public static final int DIVISION_BY_ZERO = 34;
 
-    public static final int INSTANTIATION_ERROR = 100;
+    public static final int UNSUPPORTED_JRE = 100;
 
     public static final int EXTERNAL_UDF_RESULT_TYPE_ERROR = 200;
 
@@ -130,7 +130,7 @@
     public static final int UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1047;
     public static final int DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1048;
     public static final int LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION = 1049;
-    public static final int NO_DATASET_WITH_NAME = 1050;
+    public static final int UNKNOWN_DATASET_IN_DATAVERSE = 1050;
     public static final int INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD = 1051;
     public static final int INDEX_ILLEGAL_REPETITIVE_FIELD = 1052;
     public static final int CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET = 1053;
@@ -143,13 +143,29 @@
     public static final int WITH_FIELD_MUST_BE_OF_TYPE = 1060;
     public static final int WITH_FIELD_MUST_CONTAIN_SUB_FIELD = 1061;
     public static final int MERGE_POLICY_PARAMETER_INVALID_TYPE = 1062;
-    public static final int NO_DATAVERSE_WITH_NAME = 1063;
+    public static final int UNKNOWN_DATAVERSE = 1063;
     public static final int ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION = 1064;
     public static final int CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE = 1065;
     public static final int CANNOT_SERIALIZE_A_VALUE = 1066;
     public static final int CANNOT_FIND_NON_MISSING_SELECT_OPERATOR = 1067;
     public static final int CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE = 1068;
     public static final int CANNOT_DROP_INDEX = 1069;
+    public static final int METADATA_ERROR = 1070;
+    public static final int DATAVERSE_EXISTS = 1071;
+    public static final int DATASET_EXISTS = 1072;
+    public static final int UNDEFINED_IDENTIFIER = 1073;
+    public static final int AMBIGUOUS_IDENTIFIER = 1074;
+    public static final int FORBIDDEN_SCOPE = 1075;
+    public static final int NAME_RESOLVE_UNKNOWN_DATASET = 1076;
+    public static final int NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE = 1077;
+    public static final int COMPILATION_UNEXPECTED_OPERATOR = 1078;
+    public static final int COMPILATION_ERROR = 1079;
+    public static final int UNKNOWN_NODEGROUP = 1080;
+    public static final int UNKNOWN_FUNCTION = 1081;
+    public static final int UNKNOWN_TYPE = 1082;
+    public static final int UNKNOWN_INDEX = 1083;
+    public static final int INDEX_EXISTS = 1084;
+    public static final int TYPE_EXISTS = 1085;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
index 256ce08..359054e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
@@ -20,6 +20,8 @@
 
 import java.util.function.Predicate;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public class ExceptionUtils {
     public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
     public static final String PARAMETER_NAME = "Parameter name: ";
@@ -45,13 +47,28 @@
         Throwable current = e;
         Throwable cause = e.getCause();
         while (cause != null && cause != current) {
-            Throwable nextCause = current.getCause();
             current = cause;
-            cause = nextCause;
+            cause = current.getCause();
         }
         return current;
     }
 
+    public static Throwable getCause(Throwable e, String component, int code) {
+        Throwable current = e;
+        Throwable expected =
+                (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                        && ((HyracksDataException) current).getComponent().equals(component)) ? current : null;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            expected =
+                    (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                            && ((HyracksDataException) current).getComponent().equals(component)) ? current : expected;
+            cause = current.getCause();
+        }
+        return expected == null ? current : expected;
+    }
+
     /**
      * Determines whether supplied exception contains a matching cause in its hierarchy, or is itself a match
      */
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
index 085f465..a735c81 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
@@ -21,6 +21,7 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -62,10 +63,18 @@
         super(errorCode, params);
     }
 
+    public MetadataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(errorCode, sourceLoc, params);
+    }
+
     public MetadataException(int errorCode, Throwable cause, Serializable... params) {
         super(errorCode, cause, params);
     }
 
+    public MetadataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(errorCode, cause, sourceLoc, params);
+    }
+
     public static MetadataException create(Throwable cause) {
         if (cause instanceof MetadataException || cause == null) {
             return (MetadataException) cause;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
index 85bfaa5..1c6763f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
@@ -22,6 +22,7 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class RuntimeDataException extends HyracksDataException {
     private static final long serialVersionUID = 1L;
@@ -30,7 +31,15 @@
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
     }
 
+    public RuntimeDataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), null, sourceLoc, params);
+    }
+
     public RuntimeDataException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
+
+    public RuntimeDataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
deleted file mode 100644
index f027979..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.LongPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
-
-// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
-public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
-    public static final MutableArrayValueReference LSN_KEY = new MutableArrayValueReference("LSN".getBytes());
-    public static final long INVALID = -1L;
-
-    protected final ILSMIndex lsmIndex;
-    // First LSN per mutable component. TODO: move from hyracks to asterixdb
-    protected final long[] firstLSNs;
-    // A boolean array to keep track of flush operations
-    protected final boolean[] flushRequested;
-    // TODO: move from hyracks to asterixdb
-    protected final long[] mutableLastLSNs;
-    // Index of the currently flushing or next to be flushed component
-    protected int readIndex;
-    // Index of the currently being written to component
-    protected int writeIndex;
-    // Index of the memory component to be recycled
-    protected int recycleIndex;
-    // Indicates whether this index has been scheduled to flush (no matter whether succeeds or not)
-    protected boolean hasFlushed;
-    // Keep track of the component Id of the next component being activated.
-    protected ILSMComponentId[] nextComponentIds;
-
-    protected final ILSMComponentIdGenerator idGenerator;
-    protected final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-    private final Map<ILSMComponentId, Long> componentLsnMap = new HashMap<>();
-
-    public AbstractLSMIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
-        this.lsmIndex = lsmIndex;
-        this.idGenerator = idGenerator;
-        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
-        int count = lsmIndex.getNumberOfAllMemoryComponents();
-        mutableLastLSNs = new long[count];
-        firstLSNs = new long[count];
-        flushRequested = new boolean[count];
-        readIndex = 0;
-        writeIndex = 0;
-        recycleIndex = 0;
-        hasFlushed = false;
-        nextComponentIds = new ILSMComponentId[count];
-        if (count > 0) {
-            nextComponentIds[0] = idGenerator.getId();
-        }
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            /*
-             * This method was called on the scheduleFlush operation.
-             * We set the lastLSN to the last LSN for the index (the LSN for the flush log)
-             * We mark the component flushing flag
-             * We then move the write pointer to the next component and sets its first LSN to the flush log LSN
-             */
-            synchronized (this) {
-                flushRequested[writeIndex] = true;
-                writeIndex = (writeIndex + 1) % mutableLastLSNs.length;
-                // Set the firstLSN of the next component unless it is being flushed
-                if (writeIndex != readIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-
-            }
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        //TODO: Copying Filters and all content of the metadata pages for flush operation should be done here
-        if (opCtx.getNewComponent() == null) {
-            // failed operation. Nothing to do.
-            return;
-        }
-        putLSNIntoMetadata(opCtx.getNewComponent(), opCtx.getComponentsToBeMerged());
-        putComponentIdIntoMetadata(opCtx.getIoOperationType(), opCtx.getNewComponent(),
-                opCtx.getComponentsToBeMerged());
-        componentLsnMap.put(opCtx.getNewComponent().getId(), getComponentLSN(opCtx.getComponentsToBeMerged()));
-        if (opCtx.getIoOperationType() == LSMIOOperationType.MERGE) {
-            if (opCtx.getComponentsToBeMerged().isEmpty()) {
-                throw new IllegalStateException("Merge must have old components");
-            }
-            LongPointable markerLsn = LongPointable.FACTORY
-                    .createPointable(ComponentUtils.getLong(opCtx.getComponentsToBeMerged().get(0).getMetadata(),
-                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
-            opCtx.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
-        } else if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            // advance memory component indexes
-            synchronized (this) {
-                // we've already consumed the specified LSN/component id.
-                // Now we can advance to the next component
-                flushRequested[readIndex] = false;
-                // if the component which just finished flushing is the component that will be modified next,
-                // we set its first LSN to its previous LSN
-                if (readIndex == writeIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-                readIndex = (readIndex + 1) % mutableLastLSNs.length;
-            }
-        }
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // The operation was complete and the next I/O operation for the LSM index didn't start yet
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            hasFlushed = true;
-            if (opCtx.getNewComponent() != null) {
-                final Long lsn = componentLsnMap.remove(opCtx.getNewComponent().getId());
-                if (lsn == null) {
-                    throw new IllegalStateException("Unidentified flushed component: " + opCtx.getNewComponent());
-                }
-                // empty component doesn't have any files
-                final Optional<String> componentFile =
-                        opCtx.getNewComponent().getLSMComponentPhysicalFiles().stream().findAny();
-                if (componentFile.isPresent()) {
-                    final ResourceReference ref = ResourceReference.of(componentFile.get());
-                    final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
-                    indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn);
-                }
-            }
-        }
-    }
-
-    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<? extends ILSMComponent> oldComponents)
-            throws HyracksDataException {
-        newComponent.getMetadata().put(LSN_KEY, LongPointable.FACTORY.createPointable(getComponentLSN(oldComponents)));
-    }
-
-    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
-        LongPointable pointable = new LongPointable();
-        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
-        metadataPageManager.get(metadataPageManager.createMetadataFrame(), LSN_KEY, pointable);
-        return pointable.getLength() == 0 ? INVALID : pointable.longValue();
-    }
-
-    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        if (mergedComponents.isEmpty()) {
-            return null;
-        }
-        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
-                mergedComponents.get(mergedComponents.size() - 1).getId());
-
-    }
-
-    private void putComponentIdIntoMetadata(LSMIOOperationType opType, ILSMDiskComponent newComponent,
-            List<? extends ILSMComponent> oldComponents) throws HyracksDataException {
-        // the id of flushed component is set when we copy the metadata of the memory component
-        if (opType == LSMIOOperationType.MERGE) {
-            ILSMComponentId componentId = getMergedComponentId(oldComponents);
-            LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
-        }
-    }
-
-    public synchronized void updateLastLSN(long lastLSN) {
-        if (!flushRequested[writeIndex]) {
-            //if the memory component pointed by writeIndex is being flushed, we should ignore this update call
-            //since otherwise the original LSN is overwritten.
-            //Moreover, since the memory component is already being flushed, the next scheduleFlush request must fail.
-            //See https://issues.apache.org/jira/browse/ASTERIXDB-1917
-            mutableLastLSNs[writeIndex] = lastLSN;
-            if (hasFlushed || lsmIndex.isMemoryComponentsAllocated()) {
-                // we only (re)set next component id if either this index has been flushed (no matter succeed or not)
-                // or the memory component has been allocated
-                // This prevents the case where indexes in a partition are being allocated, while another partition
-                // tries to schedule flush
-                nextComponentIds[writeIndex] = idGenerator.getId();
-            }
-        }
-    }
-
-    /**
-     * Used during the recovery process to force refresh the next component id
-     */
-    public void forceRefreshNextId() {
-        nextComponentIds[writeIndex] = idGenerator.getId();
-    }
-
-    public synchronized void setFirstLSN(long firstLSN) {
-        // We make sure that this method is only called on an empty component so the first LSN is not set incorrectly
-        firstLSNs[writeIndex] = firstLSN;
-    }
-
-    public synchronized long getFirstLSN() {
-        // We make sure that this method is only called on a non-empty component so the returned LSN is meaningful
-        // The firstLSN is always the lsn of the currently being flushed component or the next
-        // to be flushed when no flush operation is on going
-        return firstLSNs[readIndex];
-    }
-
-    public synchronized boolean hasPendingFlush() {
-        for (int i = 0; i < flushRequested.length; i++) {
-            if (flushRequested[i]) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public long getComponentLSN(List<? extends ILSMComponent> diskComponents) throws HyracksDataException {
-        if (diskComponents.isEmpty()) {
-            // Implies a flush IO operation. --> moves the flush pointer
-            // Flush operation of an LSM index are executed sequentially.
-            synchronized (this) {
-                return mutableLastLSNs[readIndex];
-            }
-        }
-        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
-        long maxLSN = -1L;
-        for (ILSMComponent c : diskComponents) {
-            DiskComponentMetadata md = ((ILSMDiskComponent) c).getMetadata();
-            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
-        }
-        return maxLSN;
-    }
-
-    private synchronized ILSMComponentId getLSMComponentId() {
-        return nextComponentIds[recycleIndex];
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        ILSMComponentId componentId = getLSMComponentId();
-        component.resetId(componentId, false);
-        if (componentSwitched) {
-            recycleIndex = (recycleIndex + 1) % nextComponentIds.length;
-        }
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        if (component == lsmIndex.getCurrentMemoryComponent()) {
-            // only set the component id for the first (current) memory component
-            ILSMComponentId componentId = getLSMComponentId();
-            component.resetId(componentId, false);
-        }
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 5b9883c..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import java.io.ObjectStreamException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.common.IResource;
-
-public abstract class AbstractLSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    protected ILSMComponentIdGeneratorFactory idGeneratorFactory;
-
-    protected transient INCServiceContext ncCtx;
-
-    protected transient IResource resource;
-
-    public AbstractLSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        this.idGeneratorFactory = idGeneratorFactory;
-    }
-
-    @Override
-    public void initialize(INCServiceContext ncCtx, IResource resource) {
-        this.ncCtx = ncCtx;
-        this.resource = resource;
-    }
-
-    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
-        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
-    }
-
-    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
-        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
-    }
-
-    private void readObjectNoData() throws ObjectStreamException {
-        idGeneratorFactory = new ILSMComponentIdGeneratorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-                // used for backward compatibility
-                // if idGeneratorFactory is not set for legacy lsm indexes, we return a default
-                // component id generator which always generates the missing component id.
-                return new ILSMComponentIdGenerator() {
-                    @Override
-                    public void refresh() {
-                        // No op
-                    }
-
-                    @Override
-                    public ILSMComponentId getId() {
-                        return LSMComponentId.MISSING_COMPONENT_ID;
-                    }
-                };
-            }
-        };
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
deleted file mode 100644
index db6c609..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 97badb2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMBTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
deleted file mode 100644
index da1446b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeWithBuddyIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(lsmIndex, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
deleted file mode 100644
index 9b32345..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMBTreeWithBuddyIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeWithBuddyIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
new file mode 100644
index 0000000..ea53d68
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.ioopcallbacks;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
+public class LSMIOOperationCallback implements ILSMIOOperationCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+    public static final String KEY_FLUSH_LOG_LSN = "FlushLogLsn";
+    public static final String KEY_NEXT_COMPONENT_ID = "NextComponentId";
+    public static final String KEY_FLUSHED_COMPONENT_ID = "FlushedComponentId";
+    private static final String KEY_FIRST_LSN = "FirstLsn";
+    private static final MutableArrayValueReference KEY_METADATA_FLUSH_LOG_LSN =
+            new MutableArrayValueReference(KEY_FLUSH_LOG_LSN.getBytes());
+    public static final long INVALID_LSN = -1L;
+    private final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
+    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    protected final DatasetInfo dsInfo;
+    protected final ILSMIndex lsmIndex;
+    private long firstLsnForCurrentMemoryComponent = 0L;
+    private long persistenceLsn = 0L;
+    private int pendingFlushes = 0;
+    private Deque<ILSMComponentId> componentIds = new ArrayDeque<>();
+
+    public LSMIOOperationCallback(DatasetInfo dsInfo, ILSMIndex lsmIndex, ILSMComponentId nextComponentId,
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+        this.dsInfo = dsInfo;
+        this.lsmIndex = lsmIndex;
+        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+        componentIds.add(nextComponentId);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+            Map<String, Object> map = operation.getParameters();
+            putComponentIdIntoMetadata(operation.getNewComponent(), (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID));
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            Map<String, Object> map = operation.getParameters();
+            putLSNIntoMetadata(operation.getNewComponent(), (Long) map.get(KEY_FLUSH_LOG_LSN));
+            putComponentIdIntoMetadata(operation.getNewComponent(),
+                    ((FlushOperation) operation).getFlushingComponent().getId());
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+            List<ILSMDiskComponent> mergedComponents = operation.getAccessor().getOpContext().getComponentsToBeMerged();
+            putLSNIntoMetadata(operation.getNewComponent(), mergedComponents);
+            putComponentIdIntoMetadata(operation.getNewComponent(), mergedComponents);
+            LongPointable markerLsn =
+                    LongPointable.FACTORY.createPointable(ComponentUtils.getLong(mergedComponents.get(0).getMetadata(),
+                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
+            operation.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
+        }
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() != LSMIOOperationType.LOAD
+                && operation.getAccessor().getOpContext().getOperation() == IndexOperation.DELETE_COMPONENTS) {
+            deleteComponentsFromCheckpoint(operation);
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+                || operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+            addComponentToCheckpoint(operation);
+        }
+    }
+
+    private void addComponentToCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+        // will always update the checkpoint file even if no new component was created
+        FileReference target = operation.getTarget();
+        Map<String, Object> map = operation.getParameters();
+        final Long lsn =
+                operation.getIOOpertionType() == LSMIOOperationType.FLUSH ? (Long) map.get(KEY_FLUSH_LOG_LSN) : 0L;
+        final LSMComponentId id = (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID);
+        final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+        final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
+        indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn, id.getMaxId());
+    }
+
+    private void deleteComponentsFromCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+        // component was deleted... if a flush, do nothing.. if a merge, must update the checkpoint file
+        if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+            // Get component id of the last disk component
+            LSMComponentId mostRecentComponentId =
+                    getMostRecentComponentId(operation.getAccessor().getOpContext().getComponentsToBeMerged());
+            // Update the checkpoint file
+            FileReference target = operation.getTarget();
+            final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+            indexCheckpointManagerProvider.get(ref).setLastComponentId(mostRecentComponentId.getMaxId());
+        } else if (operation.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+            throw new IllegalStateException("Unexpected IO operation: " + operation.getIOOpertionType());
+        }
+    }
+
+    private LSMComponentId getMostRecentComponentId(Collection<ILSMDiskComponent> deletedComponents)
+            throws HyracksDataException {
+        // must sync on opTracker to ensure list of components doesn't change
+        synchronized (lsmIndex.getOperationTracker()) {
+            List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
+            if (diskComponents.isEmpty()) {
+                LOGGER.log(Level.INFO, "There are no disk components");
+                return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+            }
+            if (deletedComponents.contains(diskComponents.get(diskComponents.size() - 1))) {
+                LOGGER.log(Level.INFO, "All disk components have been deleted");
+                return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+            }
+            int mostRecentComponentIndex = 0;
+            for (int i = 0; i < diskComponents.size(); i++) {
+                if (!deletedComponents.contains(diskComponents.get(i))) {
+                    break;
+                }
+                mostRecentComponentIndex++;
+            }
+            ILSMDiskComponent mostRecentDiskComponent = diskComponents.get(mostRecentComponentIndex);
+            return (LSMComponentId) mostRecentDiskComponent.getId();
+        }
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        putLSNIntoMetadata(newComponent, getComponentLSN(oldComponents));
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, long lsn) throws HyracksDataException {
+        newComponent.getMetadata().put(KEY_METADATA_FLUSH_LOG_LSN, LongPointable.FACTORY.createPointable(lsn));
+    }
+
+    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
+        LongPointable pointable = new LongPointable();
+        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
+        metadataPageManager.get(metadataPageManager.createMetadataFrame(), KEY_METADATA_FLUSH_LOG_LSN, pointable);
+        return pointable.getLength() == 0 ? INVALID_LSN : pointable.longValue();
+    }
+
+    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        if (mergedComponents.isEmpty()) {
+            return null;
+        }
+        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
+                mergedComponents.get(mergedComponents.size() - 1).getId());
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        ILSMComponentId componentId = getMergedComponentId(oldComponents);
+        putComponentIdIntoMetadata(newComponent, componentId);
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, ILSMComponentId componentId)
+            throws HyracksDataException {
+        LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
+    }
+
+    public synchronized void setFirstLsnForCurrentMemoryComponent(long firstLsn) {
+        this.firstLsnForCurrentMemoryComponent = firstLsn;
+        if (pendingFlushes == 0) {
+            this.persistenceLsn = firstLsn;
+        }
+    }
+
+    public synchronized long getPersistenceLsn() {
+        return persistenceLsn;
+    }
+
+    public long getComponentLSN(List<ILSMDiskComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents.isEmpty()) {
+            throw new IllegalArgumentException("Can't get LSN from an empty list of disk components");
+        }
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
+        long maxLSN = -1L;
+        for (ILSMDiskComponent c : diskComponents) {
+            DiskComponentMetadata md = c.getMetadata();
+            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
+        }
+        return maxLSN;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        component.resetId(componentIds.poll(), false);
+    }
+
+    @Override
+    public synchronized void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        dsInfo.declareActiveIOOperation();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes++;
+            FlushOperation flush = (FlushOperation) operation;
+            Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+            Long flushLsn = (Long) map.get(KEY_FLUSH_LOG_LSN);
+            map.put(KEY_FIRST_LSN, firstLsnForCurrentMemoryComponent);
+            map.put(KEY_FLUSHED_COMPONENT_ID, flush.getFlushingComponent().getId());
+            componentIds.add((ILSMComponentId) map.get(KEY_NEXT_COMPONENT_ID));
+            firstLsnForCurrentMemoryComponent = flushLsn; // Advance the first lsn for new component
+        }
+    }
+
+    @Override
+    public synchronized void completed(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes--;
+            if (operation.getStatus() == LSMIOOperationStatus.SUCCESS) {
+                Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+                persistenceLsn =
+                        pendingFlushes == 0 ? firstLsnForCurrentMemoryComponent : (Long) map.get(KEY_FLUSH_LOG_LSN);
+            }
+        }
+        dsInfo.undeclareActiveIOOperation();
+    }
+
+    public synchronized boolean hasPendingFlush() {
+        return pendingFlushes > 0;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        // No Op
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..826ca50
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.ioopcallbacks;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class LSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMComponentIdGeneratorFactory idGeneratorFactory;
+
+    protected final IDatasetInfoProvider datasetInfoProvider;
+
+    protected transient INCServiceContext ncCtx;
+
+    protected transient IResource resource;
+
+    public LSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        this.idGeneratorFactory = idGeneratorFactory;
+        this.datasetInfoProvider = datasetInfoProvider;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        this.ncCtx = ncCtx;
+        this.resource = resource;
+    }
+
+    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
+    }
+
+    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        return new LSMIOOperationCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+                getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource).getCurrentComponentIndex();
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
deleted file mode 100644
index 3ba9bcd..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMInvertedIndexIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 766ef95..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMInvertedIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMInvertedIndexIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
deleted file mode 100644
index f3e80ec..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMRTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvodier) {
-        super(index, idGenerator, checkpointManagerProvodier);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 3a0afa8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMRTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMRTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
index b008f11..2c0872c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
@@ -27,31 +27,33 @@
     /**
      * Initializes the first checkpoint of an index with low watermark {@code lsn}
      *
+     * @param componentTimestamp
      * @param lsn
      * @throws HyracksDataException
      */
-    void init(long lsn) throws HyracksDataException;
+    void init(String componentTimestamp, long lsn) throws HyracksDataException;
 
     /**
-     * Called when a new LSM disk component is flushed. When called,  the index checkpoiint is updated
+     * Called when a new LSM disk component is flushed. When called, the index checkpoint is updated
      * with the latest valid {@code componentTimestamp} and low watermark {@code lsn}
      *
      * @param componentTimestamp
      * @param lsn
      * @throws HyracksDataException
      */
-    void flushed(String componentTimestamp, long lsn) throws HyracksDataException;
+    void flushed(String componentTimestamp, long lsn, long componentId) throws HyracksDataException;
 
     /**
-     * Called when a new LSM disk component is replicated from master. When called,  the index checkpoiint is updated
+     * Called when a new LSM disk component is replicated from master. When called, the index checkpoint is updated
      * with the latest valid {@code componentTimestamp} and the local lsn mapping of {@code masterLsn} is set as the
      * new low watermark.
      *
      * @param componentTimestamp
      * @param masterLsn
+     * @param componentId
      * @throws HyracksDataException
      */
-    void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException;
+    void replicated(String componentTimestamp, long masterLsn, long componentId) throws HyracksDataException;
 
     /**
      * Called when a flush log is received and replicated from master. The mapping between
@@ -89,13 +91,37 @@
      * Gets the index last valid component timestamp if the index has any components. Otherwise {@link Optional#empty()}
      *
      * @return the index last valid component timestamp
+     * @throws HyracksDataException
      */
-    Optional<String> getValidComponentTimestamp();
+    Optional<String> getValidComponentTimestamp() throws HyracksDataException;
 
     /**
      * Gets the number of valid checkpoints the index has.
      *
      * @return the number of valid checkpoints
+     * @throws HyracksDataException
      */
-    int getCheckpointCount();
+    int getCheckpointCount() throws HyracksDataException;
+
+    /**
+     * @return the latest checkpoint
+     * @throws HyracksDataException
+     */
+    IndexCheckpoint getLatest() throws HyracksDataException;
+
+    /**
+     * Advance the last valid component timestamp. Used for replicated bulkloaded components
+     *
+     * @param timeStamp
+     * @throws HyracksDataException
+     */
+    void advanceValidComponentTimestamp(String timeStamp) throws HyracksDataException;
+
+    /**
+     * Set the last component id. Used during recovery or after component delete
+     *
+     * @param componentId
+     * @throws HyracksDataException
+     */
+    void setLastComponentId(long componentId) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index 6e845e1..73d3122 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.core.JsonProcessingException;
@@ -35,24 +36,28 @@
     private long id;
     private String validComponentTimestamp;
     private long lowWatermark;
+    private long lastComponentId;
     private Map<Long, Long> masterNodeFlushMap;
 
-    public static IndexCheckpoint first(long lowWatermark) {
+    public static IndexCheckpoint first(String lastComponentTimestamp, long lowWatermark) {
         IndexCheckpoint firstCheckpoint = new IndexCheckpoint();
         firstCheckpoint.id = INITIAL_CHECKPOINT_ID;
         firstCheckpoint.lowWatermark = lowWatermark;
-        firstCheckpoint.validComponentTimestamp = null;
+        firstCheckpoint.validComponentTimestamp = lastComponentTimestamp;
+        firstCheckpoint.lastComponentId = LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId();
         firstCheckpoint.masterNodeFlushMap = new HashMap<>();
         return firstCheckpoint;
     }
 
-    public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, String validComponentTimestamp) {
+    public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, String validComponentTimestamp,
+            long lastComponentId) {
         if (lowWatermark < latest.getLowWatermark()) {
             throw new IllegalStateException("Low watermark should always be increasing");
         }
         IndexCheckpoint next = new IndexCheckpoint();
         next.id = latest.getId() + 1;
         next.lowWatermark = lowWatermark;
+        next.lastComponentId = lastComponentId;
         next.validComponentTimestamp = validComponentTimestamp;
         next.masterNodeFlushMap = latest.getMasterNodeFlushMap();
         // remove any lsn from the map that wont be used anymore
@@ -72,6 +77,10 @@
         return lowWatermark;
     }
 
+    public long getLastComponentId() {
+        return lastComponentId;
+    }
+
     public Map<Long, Long> getMasterNodeFlushMap() {
         return masterNodeFlushMap;
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
deleted file mode 100644
index 82bfdfa..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.transactions;
-
-/**
- * @author pouria Interface for a hashTable, used in the internal data
- *         structures of lockManager
- * @param <K>
- *            Type of the objects, used as keys
- * @param <V>
- *            Type of the objects, used as values
- */
-public interface ILockHashTable<K, V> {
-
-    public void put(K key, V value);
-
-    public V get(K key);
-
-    public V remove(K key);
-
-    public int getKeysetSize();
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
deleted file mode 100644
index a545ac2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.transactions;
-
-/**
- * @author pouria
- *         Shows: - The conflict matrix for the locking protocol (whether two
- *         lock modes conflict with each other or not on a single resource) -
- *         Whether request to convert a lock mode to a new one is a conversion
- *         (i.e. the new lock mode is stringer than the current one) or not
- *         Each lock mode is shown/interpreted as an integer
- */
-
-public interface ILockMatrix {
-
-    /**
-     * @param mask
-     *            (current/expected) lock mask on the resource
-     * @param reqLockMode
-     *            index of the requested lockMode
-     * @return true if the lock request conflicts with the mask
-     */
-    public boolean conflicts(int mask, int reqLockMode);
-
-    /**
-     * @param currentLockMode
-     * @param reqLockMode
-     * @return true if the request is a conversion
-     */
-    public boolean isConversion(int currentLockMode, int reqLockMode);
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
deleted file mode 100644
index b5ee0a8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.transactions;
-
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public interface ILongBlockFactory {
-    /**
-     * Ensures future blocks are allocated larger than the supplied value
-     *
-     * @param value
-     *            the value to ensure future blocks are larger than
-     */
-    void ensureMinimum(long value) throws AlgebricksException;
-
-    /**
-     * Allocates a block of longs of specified block size
-     *
-     * @param blockSize
-     *            The size of the block of longs to reserve
-     * @return the start of the reserved block
-     */
-    long getBlock(int blockSize) throws AlgebricksException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java
new file mode 100644
index 0000000..1da5c9c
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.transactions;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public interface IRecoveryManagerFactory {
+
+    /**
+     * Create the local recovery manager
+     *
+     * @param serviceCtx
+     *            the service context
+     * @param txnSubsystem
+     *            the transaction subsystem
+     * @return the recovery manager
+     */
+    IRecoveryManager createRecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
deleted file mode 100644
index 94ca848..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.transactions;
-
-import java.io.Serializable;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-
-public interface ITxnIdBlockProvider extends Remote, Serializable {
-
-    /**
-     * Ensures that future transaction blocks will be of a value larger than the supplied value
-     *
-     * @param maxId
-     *            The txn id to ensure future txn ids are larger than
-     * @throws RemoteException
-     */
-    void ensureMinimumTxnId(long maxId) throws RemoteException;
-
-    /**
-     * Allocates a block of transaction ids of specified block size
-     *
-     * @param blockSize
-     *            The size of the transaction id block to reserve
-     * @return the start of the reserved block
-     * @throws RemoteException
-     */
-    long reserveTxnIdBlock(int blockSize) throws RemoteException;
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 5fdb4e2..0c3b21d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -24,7 +24,6 @@
 import java.util.zip.CRC32;
 
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
-import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -271,6 +270,7 @@
                 computeAndSetLogSize();
                 break;
             case LogType.WAIT:
+            case LogType.WAIT_FOR_FLUSHES:
                 computeAndSetLogSize();
                 break;
             case LogType.JOB_COMMIT:
@@ -462,6 +462,7 @@
                 logSize = FLUSH_LOG_SIZE;
                 break;
             case LogType.WAIT:
+            case LogType.WAIT_FOR_FLUSHES:
                 logSize = WAIT_LOG_SIZE;
                 break;
             case LogType.FILTER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
index f02b0de..2d76a11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
@@ -28,6 +28,7 @@
     public static final byte WAIT = 6;
     public static final byte FILTER = 7;
     public static final byte MARKER = 8;
+    public static final byte WAIT_FOR_FLUSHES = 9;
 
     private static final String STRING_UPDATE = "UPDATE";
     private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
@@ -37,6 +38,7 @@
     private static final String STRING_WAIT = "WAIT";
     private static final String STRING_FILTER = "FILTER";
     private static final String STRING_MARKER = "MARKER";
+    private static final String STRING_WAIT_FOR_FLUSHES = "WAIT_FOR_FLUSHES";
     private static final String STRING_UNKNOWN_LOG_TYPE = "UNKNOWN_LOG_TYPE";
 
     public static String toString(byte logType) {
@@ -53,6 +55,8 @@
                 return STRING_FLUSH;
             case LogType.WAIT:
                 return STRING_WAIT;
+            case LogType.WAIT_FOR_FLUSHES:
+                return STRING_WAIT_FOR_FLUSHES;
             case LogType.FILTER:
                 return STRING_FILTER;
             case LogType.MARKER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
index 1ac3ffa..b135c7f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
@@ -20,14 +20,6 @@
 
 public class Servlets {
 
-    public static final String AQL = "/aql";
-    public static final String AQL_QUERY = "/query";
-    public static final String AQL_UPDATE = "/update";
-    public static final String AQL_DDL = "/ddl";
-    public static final String SQLPP = "/sqlpp";
-    public static final String SQLPP_QUERY = "/query/sqlpp";
-    public static final String SQLPP_UPDATE = "/update/sqlpp";
-    public static final String SQLPP_DDL = "/ddl/sqlpp";
     public static final String QUERY_STATUS = "/query/service/status/*";
     public static final String QUERY_RESULT = "/query/service/result/*";
     public static final String QUERY_SERVICE = "/query/service";
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
index 265c9fd..4bde9cf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
@@ -37,23 +37,17 @@
     public static final String METADATA_FILE_NAME = ".metadata";
     public static final String MASK_FILE_PREFIX = ".mask_";
     public static final String COMPONENT_MASK_FILE_PREFIX = MASK_FILE_PREFIX + "C_";
-    public static final String LEGACY_DATASET_INDEX_NAME_SEPARATOR = "_idx_";
 
     /**
      * The storage version of AsterixDB related artifacts (e.g. log files, checkpoint files, etc..).
      */
-    private static final int LOCAL_STORAGE_VERSION = 2;
+    private static final int LOCAL_STORAGE_VERSION = 3;
 
     /**
      * The storage version of AsterixDB stack.
      */
     public static final int VERSION = LOCAL_STORAGE_VERSION + ITreeIndexFrame.Constants.VERSION;
 
-    /**
-     * The storage version in which the rebalance storage structure was introduced
-     */
-    public static final int REBALANCE_STORAGE_VERSION = 8;
-
     private StorageConstants() {
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 6b13468..aa2c7af 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -19,17 +19,17 @@
 package org.apache.asterix.common.utils;
 
 import java.io.File;
+import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.function.Function;
 
 import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.storage.IndexPathElements;
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.MappedFileSplit;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -143,4 +143,16 @@
     public static String getIndexNameFromPath(String path) {
         return Paths.get(path).getFileName().toString();
     }
+
+    /**
+     * Get the path of the index containing the passed reference
+     *
+     * @param ioManager
+     * @param ref
+     * @return
+     * @throws HyracksDataException
+     */
+    public static Path getIndexPath(IIOManager ioManager, ResourceReference ref) throws HyracksDataException {
+        return ioManager.resolve(ref.getRelativePath().toString()).getFile().toPath();
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index a0cebea..f0a36bf 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -69,7 +69,7 @@
 33 = Node is not registered with the CC
 34 = Division by Zero.
 
-100 = Unable to instantiate class %1$s
+100 = Unsupported JRE: %1$s
 
 200 = External UDF cannot produce expected result. Please check the UDF configuration
 
@@ -117,7 +117,7 @@
 1047 = Metadata lock cannot be upgraded! because it was not acquired before
 1048 = Metadata lock cannot be downgraded! because it was not acquired before
 1049 = Metadata lock cannot be acquired for %1$s since it is already acquired for %2$s
-1050 = There is no dataset with name %2$s in dataverse %1$s
+1050 = Cannot find dataset with name %1$s in dataverse %2$s
 1051 = Cannot create enforced index on \"%1$s\" field. The field is closed type.
 1052 = Cannot create index with the same field \"%1$s\" specified more than once.
 1053 = Cannot create primary index on external dataset.
@@ -130,13 +130,29 @@
 1060 = Field \"%1$s\" in the with clause must be of type %2$s
 1061 = Field \"%1$s\" in the with clause must contain sub field \"%2$s\"
 1062 = Merge policy parameters cannot be of type %1$s
-1063 = There is no dataverse with name \"%1$s\"
+1063 = Cannot find dataverse with name %1$s
 1064 = An error was occurred while converting type %1$s to type %2$s.
 1065 = There should be at least two applicable indexes.
 1066 = Cannot serialize a value.
 1067 = Cannot find a non-missing SELECT operator in GROUP operator for a left-outer-join plan optimization.
 1068 = Cannot get the conditional split variable for the given UNNESTMAP operator.
 1069 = Cannot drop index \"%1$s\". Drop dataset \"%1$s\" to remove this index
+1070 = Metadata error. %1$s
+1071 = A dataverse with this name %1$s already exists.
+1072 = A dataset with this name %1$s already exists.
+1073 = Cannot resolve alias reference for undefined identifier %1$s in %2$s
+1074 = Cannot resolve ambiguous alias reference for undefined identifier %1$s in %2$s
+1075 = Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.
+1076 = Cannot find dataset %1$s because there is no dataverse declared, nor an alias with name %1$s!
+1077 = Cannot find dataset %1$s in dataverse %2$s nor an alias with name %1$s!
+1078 = Unexpected operator %1$s in an OperatorExpr starting with %2$s
+1079 = %1$s
+1080 = Cannot find node group with name %1$s
+1081 = Cannot find function with name %1$s
+1082 = Cannot find datatype wit name %1$s
+1083 = Cannot find index with name %1$s
+1084 = An index with this name %1$s already exists
+1085 = A datatype with this name %1$s already exists
 
 # Feed Errors
 3001 = Illegal state.
@@ -211,7 +227,7 @@
 3071 = Found END_RECORD while expecting a list item
 3072 = Cannot cast the %1$s type to the %2$s type
 3073 = Missing deserializer method for constructor: %1$s
-3074 = This can not be an instance of %1$s
+3074 = %1$s cannot be an instance of %2$s
 3075 = Closed field %1$s has null value
 3076 = %1$s: no files found
 3077 = %1$s: path not found
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index befbeed..843cb21 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -220,7 +219,7 @@
 
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 mergedComponents.forEach(component -> {
                     try {
                         resultComponentIDs.add(component.getId());
@@ -230,8 +229,7 @@
                 });
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
         Mockito.when(index.isPrimaryIndex()).thenReturn(isPrimary);
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
deleted file mode 100644
index c03af40..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import java.util.Collections;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-
-import junit.framework.TestCase;
-
-public abstract class AbstractLSMIOOperationCallbackTest extends TestCase {
-
-    @Test
-    public void testNormalSequence() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-
-        //request to flush first component
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        //request to flush second component
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testOverWrittenLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        //request to flush first component again
-        //this call should fail
-        callback.updateLastLSN(3);
-        //there is no corresponding beforeOperation, since the first component is being flush
-        //the scheduleFlush request would fail this time
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testLostLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-
-        // the first flush is finished, but has not finalized yet (in codebase, these two calls
-        // are not synchronized)
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-
-        //request to flush first component again
-        callback.updateLastLSN(3);
-
-        // the first flush is finalized (it may be called after afterOperation for a while)
-        callback.afterFinalize(firstOpCtx);
-
-        // the second flush gets LSN 2
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        // the second flush is finished
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-
-        // it should get new LSN 3
-        Assert.assertEquals(3, callback.getComponentLSN(Collections.emptyList()));
-    }
-
-    @Test
-    public void testAllocateComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId initialId = idGenerator.getId();
-        // simulate a partition is flushed before allocated
-        idGenerator.refresh();
-        callback.updateLastLSN(0);
-
-        callback.allocated(mockComponent);
-        checkMemoryComponent(initialId, mockComponent);
-    }
-
-    @Test
-    public void testRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 100; i++) {
-            // schedule a flush
-            idGenerator.refresh();
-            ILSMComponentId expectedId = idGenerator.getId();
-            callback.updateLastLSN(0);
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.beforeOperation(opCtx);
-            callback.recycled(mockComponent, true);
-            opCtx.setNewComponent(mockDiskComponent());
-            callback.afterOperation(opCtx);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(expectedId, mockComponent);
-        }
-    }
-
-    @Test
-    public void testRecycleWithoutSwitch() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 10; i++) {
-            idGenerator.refresh();
-            id = idGenerator.getId();
-            callback.updateLastLSN(0);
-            // Huh! There is no beforeOperation?
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.recycled(mockComponent, false);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(id, mockComponent);
-        }
-    }
-
-    @Test
-    public void testConcurrentRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-
-        // schedule a flush
-        idGenerator.refresh();
-        ILSMComponentId expectedId = idGenerator.getId();
-
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        // another flush is to be scheduled before the component is recycled
-        idGenerator.refresh();
-        ILSMComponentId nextId = idGenerator.getId();
-
-        // recycle the component
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(expectedId, mockComponent);
-
-        // schedule the next flush
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(nextId, mockComponent);
-    }
-
-    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
-            throws HyracksDataException {
-        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
-        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
-        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
-        assertEquals(expected, idArgument.getValue());
-        assertEquals(false, forceArgument.getValue().booleanValue());
-
-        Mockito.reset(memoryComponent);
-    }
-
-    private ILSMDiskComponent mockDiskComponent() {
-        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
-        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
-        return component;
-    }
-
-    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
-        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
-                Mockito.mock(IIndexCheckpointManagerProvider.class);
-        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
-        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong());
-        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
-        return indexCheckpointManagerProvider;
-    }
-
-    protected abstract AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
deleted file mode 100644
index a4bc399..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
deleted file mode 100644
index 5f37c78..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeWithBuddyIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeWithBuddyIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
new file mode 100644
index 0000000..29a2aa0
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.test.ioopcallbacks;
+
+import java.text.Format;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import junit.framework.TestCase;
+
+public class LSMIOOperationCallbackTest extends TestCase {
+    /*
+     * The normal sequence of calls:
+     * 1. refresh id generator
+     * 2. flushLsn
+     * 3. created
+     * 4. before
+     * 5. after
+     * 6. finalize
+     * 7. destroy
+     */
+
+    private static final Format FORMATTER =
+            new SimpleDateFormat(AbstractLSMIndexFileManager.COMPONENT_TIMESTAMP_FORMAT);
+
+    private static String getComponentFileName() {
+        Date date = new Date();
+        String ts = FORMATTER.format(date);
+        return ts + '_' + ts;
+    }
+
+    @Test
+    public void testNormalSequence() throws HyracksDataException {
+        int numMemoryComponents = 2;
+
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        String indexId = "mockIndexId";
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        LSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        //Flush first
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor firstAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        firstAccessor.getOpContext().setParameters(flushMap);
+        FileReference firstTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+        LSMComponentFileReferences firstFiles = new LSMComponentFileReferences(firstTarget, firstTarget, firstTarget);
+        FlushOperation firstFlush = new TestFlushOperation(firstAccessor, firstTarget, callback, indexId, firstFiles,
+                new LSMComponentId(0, 0));
+        callback.scheduled(firstFlush);
+        callback.beforeOperation(firstFlush);
+
+        //Flush second
+        idGenerator.refresh();
+        flushLsn = 2L;
+        nextComponentId = idGenerator.getId();
+        flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor secondAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        secondAccessor.getOpContext().setParameters(flushMap);
+        FileReference secondTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+        LSMComponentFileReferences secondFiles =
+                new LSMComponentFileReferences(secondTarget, secondTarget, secondTarget);
+        FlushOperation secondFlush = new TestFlushOperation(secondAccessor, secondTarget, callback, indexId,
+                secondFiles, new LSMComponentId(1, 1));
+        callback.scheduled(secondFlush);
+        callback.beforeOperation(secondFlush);
+
+        Map<String, Object> firstFlushMap = firstFlush.getAccessor().getOpContext().getParameters();
+        long firstFlushLogLsn = (Long) firstFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(1, firstFlushLogLsn);
+        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
+        firstFlush.setNewComponent(diskComponent1);
+        callback.afterOperation(firstFlush);
+        callback.afterFinalize(firstFlush);
+        callback.completed(firstFlush);
+
+        Map<String, Object> secondFlushMap = secondFlush.getAccessor().getOpContext().getParameters();
+        long secondFlushLogLsn = (Long) secondFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(2, secondFlushLogLsn);
+        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
+        secondFlush.setNewComponent(diskComponent2);
+        callback.afterOperation(secondFlush);
+        callback.afterFinalize(secondFlush);
+        callback.completed(secondFlush);
+    }
+
+    @Test
+    public void testAllocateComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        ILSMComponentId initialId = idGenerator.getId();
+        // simulate a partition is flushed before allocated
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        callback.allocated(mockComponent);
+        callback.recycled(mockComponent);
+        checkMemoryComponent(initialId, mockComponent);
+    }
+
+    @Test
+    public void testRecycleComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        String indexId = "mockIndexId";
+        ILSMComponentId id = idGenerator.getId();
+        callback.recycled(mockComponent);
+        checkMemoryComponent(id, mockComponent);
+
+        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
+        for (int i = 0; i < 100; i++) {
+            // schedule a flush
+            idGenerator.refresh();
+            ILSMComponentId expectedId = idGenerator.getId();
+            long flushLsn = 0L;
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, expectedId);
+            ILSMIndexAccessor accessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+            accessor.getOpContext().setParameters(flushMap);
+            FileReference target = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+            LSMComponentFileReferences files = new LSMComponentFileReferences(target, target, target);
+            FlushOperation flush =
+                    new TestFlushOperation(accessor, target, callback, indexId, files, new LSMComponentId(0, 0));
+            callback.scheduled(flush);
+            callback.beforeOperation(flush);
+            callback.recycled(mockComponent);
+            flush.setNewComponent(mockDiskComponent());
+            callback.afterOperation(flush);
+            callback.afterFinalize(flush);
+            callback.completed(flush);
+            checkMemoryComponent(expectedId, mockComponent);
+        }
+    }
+
+    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
+            throws HyracksDataException {
+        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
+        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
+        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
+        assertEquals(expected, idArgument.getValue());
+        assertEquals(false, forceArgument.getValue().booleanValue());
+        Mockito.reset(memoryComponent);
+    }
+
+    private ILSMDiskComponent mockDiskComponent() {
+        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
+        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
+        return component;
+    }
+
+    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
+        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
+                Mockito.mock(IIndexCheckpointManagerProvider.class);
+        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
+        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong(), Mockito.anyLong());
+        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
+        return indexCheckpointManagerProvider;
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
deleted file mode 100644
index 343bc59..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMInvertedIndexIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMInvertedIndexIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
deleted file mode 100644
index 10d95d8..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMRTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMRTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
new file mode 100644
index 0000000..8a02aa3
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.ioopcallbacks;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.mockito.Mockito;
+
+public class TestFlushOperation extends FlushOperation {
+
+    private final LSMComponentFileReferences files;
+    private final ILSMMemoryComponent flushingComponent;
+
+    public TestFlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
+            String indexIdentifier, LSMComponentFileReferences files, LSMComponentId componentId)
+            throws HyracksDataException {
+        super(accessor, target, callback, indexIdentifier);
+        this.files = files;
+        flushingComponent = accessor.getOpContext().getIndex().getCurrentMemoryComponent();
+        Mockito.when(flushingComponent.getId()).thenReturn(componentId);
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return files;
+    }
+
+    @Override
+    public ILSMComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
new file mode 100644
index 0000000..c2621d8
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.ioopcallbacks;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+public class TestLSMIndexAccessor implements ILSMIndexAccessor {
+
+    private final ILSMIndexOperationContext opCtx;
+
+    public TestLSMIndexAccessor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IIndexCursor createSearchCursor(boolean exclusive) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+    }
+
+    @Override
+    public ILSMIndexOperationContext getOpContext() {
+        return opCtx;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpdateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scanDiskComponents(IIndexCursor cursor) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateFilter(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
index 9b749fa..4306dda 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -28,10 +29,9 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -44,8 +44,8 @@
     private final List<ILSMDiskComponent> componentsToBeReplicated = new ArrayList<>();
     private boolean isAccessingComponents;
     private IndexOperation op;
-    private LSMIOOperationType ioOperationType;
-    private ILSMDiskComponent newComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> map;
     private boolean filterSkip = false;
     private boolean isRecovery = false;
 
@@ -180,22 +180,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOperationType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOperationType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.map = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newComponent = component;
+    public Map<String, Object> getParameters() {
+        return map;
     }
+
 }
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
index ac2a465..dfb1b0b 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
@@ -76,6 +76,28 @@
         { "v1": false, "v2": true, "v3": null }
 
 
+### random ###
+ * Syntax:
+
+        random( [seed_value] )
+
+ * Returns a random number, accepting an optional seed value
+ * Arguments:
+    * `seed_value`: an optional `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value representing the seed number.
+ * Return Value:
+    * A random number of type `double` between 0 and 1,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        {
+          "v1": random(),
+          "v2": random(unix_time_from_datetime_in_ms(current_datetime()))
+        };
+
+
 ### range ###
  * Syntax:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
index ed9551e..f0e98fd 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
@@ -199,3 +199,150 @@
         { "a": true, "b": true, "c": true, "d": 1, "e": true, "f": true, "g": 1 }
 
  The function has an alias `ifnanorinf`.
+
+
+### null_if (nullif) ###
+
+ * Syntax:
+
+        null_if(expression1, expression2)
+
+ * Compares two arguments and returns `null` if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if
+        * any argument is a `null` value but no argument is a `missing` value, or
+        * `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": null_if("asterixdb", "asterixdb"),
+            "b": null_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": null, "b": 1 }
+
+ The function has an alias `nullif`.
+
+
+### missing_if (missingif) ###
+
+ * Syntax:
+
+        missing_if(expression1, expression2)
+
+ * Compares two arguments and returns `missing` if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if
+        * any argument is a `missing` value, or
+        * no argument is a `null` value and `argument1` = `argument2`
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": missing_if("asterixdb", "asterixdb")
+            "b": missing_if(1, 2),
+        };
+
+ * The expected result is:
+
+        { "b": 1 }
+
+ The function has an alias `missingif`.
+
+
+### nan_if (nanif) ###
+
+ * Syntax:
+
+        nan_if(expression1, expression2)
+
+ * Compares two arguments and returns `NaN` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `NaN` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(nan_if("asterixdb", "asterixdb")),
+            "b": nan_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "NaN", "b": 1 }
+
+ The function has an alias `nanif`.
+
+
+### posinf_if (posinfif) ###
+
+ * Syntax:
+
+        posinf_if(expression1, expression2)
+
+ * Compares two arguments and returns `+INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `+INF` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(posinf_if("asterixdb", "asterixdb")),
+            "b": posinf_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "+INF", "b": 1 }
+
+ The function has an alias `posinfif`.
+
+
+### neginf_if (neginfif) ###
+
+ * Syntax:
+
+        neginf_if(expression1, expression2)
+
+ * Compares two arguments and returns `-INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `-INF` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(neginf_if("asterixdb", "asterixdb")),
+            "b": neginf_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "-INF", "b": 1 }
+
+ The function has an alias `neginfif`.
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
index c058d10..b95e778c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
@@ -198,6 +198,47 @@
         { "v1": 0.5403023058681398, "v2": -0.4161468365471424, "v3": 1.0, "v4": 0.8775825618903728, "v5": 0.562379076290703 }
 
 
+### degrees ###
+ * Syntax:
+
+        degrees(numeric_value)
+
+ * Converts radians to degrees
+ * Arguments:
+    * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+    * The degrees value for the given radians value. The returned value has type `double`,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        { "v1": degrees(pi()) };
+
+
+ * The expected result is:
+
+        { "v1": 180.0 }
+
+
+### e ###
+ * Syntax:
+
+        e()
+
+ * Return Value:
+    * e (base of the natural logarithm)
+
+ * Example:
+
+        { "v1": e() };
+
+ * The expected result is:
+
+        { "v1": 2.718281828459045 }
+
+
 ### exp ###
  * Syntax:
 
@@ -301,6 +342,22 @@
         { "v1": 0.0, "v2": 0.3010299956639812, "v3": "-Infinity", "v4": -0.3010299956639812, "v5": 3.0 }
 
 
+### pi ###
+ * Syntax:
+
+        pi()
+
+ * Return Value:
+    * Pi
+
+ * Example:
+
+        { "v1": pi() };
+
+ * The expected result is:
+
+        { "v1": 3.141592653589793 }
+
 
 ### power ###
  * Syntax:
@@ -327,6 +384,30 @@
         { "v1": 1, "v3": 0, "v4": 1.4142135623730951 }
 
 
+### radians ###
+ * Syntax:
+
+        radians(numeric_value)
+
+ * Converts degrees to radians
+ * Arguments:
+    * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+    * The radians value for the given degrees value. The returned value has type `double`,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        { "v1": radians(180) };
+
+
+ * The expected result is:
+
+        { "v1": 3.141592653589793 }
+
+
 ### round ###
  * Syntax:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
index 0e548a7..6599dc3 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
@@ -414,6 +414,30 @@
           "v2": "product-a and x-phone"
         }
 
+### reverse ###
+ * Syntax:
+
+        reverse(string)
+
+ * Returns a string formed by reversing characters in the input `string`.
+ * Arguments:
+    * `string` : a `string` to be reversed
+ * Return Value:
+    * a string containing characters from the the input `string` in the reverse order,
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value,
+    * a type error will be raised if:
+        * the first argument is any other non-string value
+
+ * Example:
+
+        reverse("hello");
+
+
+ * The expected result is:
+
+        "olleh"
+
 ### rtrim ###
  * Syntax:
 
@@ -504,7 +528,7 @@
  * Arguments:
     * `string` : a `string` to be extracted,
     * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`
-                 (starting at 0),
+                 (starting at 0). If negative then counted from the end of the string,
     * `length` : (Optional) an an `tinyint`/`smallint`/`integer`/`bigint` value as the length of the substring.
  * Return Value:
     * a `string` that represents the substring,
@@ -524,6 +548,7 @@
 
         "str"
 
+The function has an alias `substring`.
 
 ### trim ###
  * Syntax:
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md b/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
index fefdb7b..f3c256e 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
@@ -287,3 +287,100 @@
 
         [ "id", "project", "address" ]
 
+### object_remove ###
+ * Syntax:
+
+        object_remove(input_object, field_name)
+
+ * Returns a new object that has the same fields as the input object except the field to be removed
+ * Arguments:
+    * `input_object` : an object value.
+    * `field_name` : a string field name.
+ * Return Value:
+    * A new object that has the same fields as `input_object` except the field `field_name`,
+    * `missing` if the argument `input_object` or `field_name` is missing,
+    * `null` if the argument `input_object` is `null` or any other non-object value, or the argument `field_name`
+       is `null` or any other non-string value.
+
+ * Example:
+
+        object_remove(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "address"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "AsterixDB",
+        }
+
+### object_rename ###
+ * Syntax:
+
+        object_name(input_object, old_field, new_field)
+
+ * Returns a new object that has the same fields as `input_object` with field `old_field` replaced by `new_field`
+ * Arguments:
+    * `input_object` : an object value.
+    * `old_field` : a string representing the old (original) field name inside the object `input_object`.
+    * `new_field` : a string representing the new field name to replace `old_field` inside the object `input_object`.
+ * Return Value:
+    * A new object that has the same fields as `input_object` with field `old_field` replaced by `new_field`,
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is `null` or `input_object` is non-object value, or `old_field` is non-string value, or
+      `new_field` is any non-string value.
+
+ * Example:
+
+        object_rename(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "address"
+                       , "location"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "AsterixDB",
+          "location": {"city": "Irvine", "state": "CA"}
+        }
+
+### object_wrap ###
+ * Syntax:
+
+        object_wrap(input_object)
+
+ * Returns the value of the single name-value pair that appears in `input_object`.
+ * Arguments:
+    * `input_object` : an object value that consists of exactly one name-value pair.
+ * Return Value:
+    * The value of the single name-value pair that appears in `input_object`,
+    * `missing` if `input_object` is `missing`,
+    * `null` if `input_object` is null, or an empty object, or there is more than one name-value pair in `input_object`,
+      or any non-object value.
+
+ * Example:
+
+        object_wrap(
+                     {
+                       "id": 1
+                     }
+                   );
+
+ * The expected result is:
+
+        {
+          1
+        }
+
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
index 90eb017..921bc7e 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
@@ -168,10 +168,10 @@
         3.4
 
 
-### coll_count ###
+### strict_count ###
  * Syntax:
 
-        coll_count(collection)
+        strict_count(collection)
 
  * Gets the number of items in the given collection.
  * Arguments:
@@ -185,16 +185,16 @@
 
  * Example:
 
-        coll_count( [1, 2, null, missing] );
+        strict_count( [1, 2, null, missing] );
 
  * The expected result is:
 
         4
 
-### coll_avg ###
+### strict_avg ###
  * Syntax:
 
-        coll_avg(num_collection)
+        strict_avg(num_collection)
 
  * Gets the average value of the numeric items in the given collection.
  * Arguments:
@@ -210,16 +210,16 @@
 
  * Example:
 
-        coll_avg( [100, 200, 300] );
+        strict_avg( [100, 200, 300] );
 
  * The expected result is:
 
         [ 200.0 ]
 
-### coll_sum ###
+### strict_sum ###
  * Syntax:
 
-        coll_sum(num_collection)
+        strict_sum(num_collection)
 
  * Gets the sum of the items in the given collection.
  * Arguments:
@@ -237,7 +237,7 @@
 
  * Example:
 
-        coll_sum( [100, 200, 300] );
+        strict_sum( [100, 200, 300] );
 
  * The expected result is:
 
@@ -246,7 +246,7 @@
 ### array_min ###
  * Syntax:
 
-        coll_min(num_collection)
+        strict_min(num_collection)
 
  * Gets the min value of comparable items in the given collection.
  * Arguments:
@@ -265,7 +265,7 @@
 
  * Example:
 
-        coll_min( [10.2, 100, 5] );
+        strict_min( [10.2, 100, 5] );
 
  * The expected result is:
 
@@ -275,7 +275,7 @@
 ### array_max ###
  * Syntax:
 
-        coll_max(num_collection)
+        strict_max(num_collection)
 
  * Gets the max value of numeric items in the given collection.
  * Arguments:
@@ -294,7 +294,7 @@
 
  * Example:
 
-        coll_max( [10.2, 100, 5] );
+        strict_max( [10.2, 100, 5] );
 
  * The expected result is:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
index 0569667..20c81a0 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
@@ -26,7 +26,6 @@
            * [Collection Operators](#Collection_operators)
            * [Comparison Operators](#Comparison_operators)
            * [Logical Operators](#Logical_operators)
-      * [Case Expressions](#Case_expressions)
       * [Quantified Expressions](#Quantified_expressions)
       * [Path Expressions](#Path_expressions)
       * [Primary Expressions](#Primary_expressions)
@@ -34,6 +33,7 @@
            * [Variable References](#Variable_references)
            * [Parenthesized Expressions](#Parenthesized_expressions)
            * [Function call Expressions](#Function_call_expressions)
+           * [Case Expressions](#Case_expressions)
            * [Constructors](#Constructors)
 * [3. Queries](#Queries)
       * [Declarations](#Declarations)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index 5f407ca..8fe379d 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -19,11 +19,10 @@
 
 SQL++ is a highly composable expression language. Each SQL++ expression returns zero or more data model instances.
 There are three major kinds of expressions in SQL++. At the topmost level, a SQL++ expression can be an
-OperatorExpression (similar to a mathematical expression), an ConditionalExpression (to choose between
-alternative values), or a QuantifiedExpression (which yields a boolean value). Each will be detailed as we
-explore the full SQL++ grammar.
+OperatorExpression (similar to a mathematical expression), or a QuantifiedExpression (which yields a boolean value). 
+Each will be detailed as we explore the full SQL++ grammar.
 
-    Expression ::= OperatorExpression | CaseExpression | QuantifiedExpression
+    Expression ::= OperatorExpression | QuantifiedExpression
 
 Note that in the following text, words enclosed in angle brackets denote keywords that are not case-sensitive.
 
@@ -50,9 +49,9 @@
 |-----------------------------------------------------------------------------|-----------|
 | EXISTS, NOT EXISTS                                                          |  Collection emptiness testing |
 | ^                                                                           |  Exponentiation  |
-| *, /, %                                                                     |  Multiplication, division, modulo |
+| *, /, DIV, MOD (%)                                                          |  Multiplication, division, modulo |
 | +, -                                                                        |  Addition, subtraction  |
-| &#124;&#124;                                                                          |  String concatenation |
+| &#124;&#124;                                                                |  String concatenation |
 | IS NULL, IS NOT NULL, IS MISSING, IS NOT MISSING, <br/>IS UNKNOWN, IS NOT UNKNOWN, IS VALUED, IS NOT VALUED | Unknown value comparison |
 | BETWEEN, NOT BETWEEN                                                        | Range comparison (inclusive on both sides) |
 | =, !=, <>, <, >, <=, >=, LIKE, NOT LIKE, IN, NOT IN                             | Comparison  |
@@ -72,7 +71,10 @@
 |--------------|-------------------------------------------------------------------------|------------|
 | +, -         |  As unary operators, they denote a <br/>positive or negative expression | SELECT VALUE -1; |
 | +, -         |  As binary operators, they add or subtract                              | SELECT VALUE 1 + 2; |
-| *, /, %      |  Multiply, divide, modulo                                               | SELECT VALUE 4 / 2.0; |
+| *            |  Multiply                                                               | SELECT VALUE 4 * 2; |
+| /            |  Divide (returns a value of type `double` if both operands are integers)| SELECT VALUE 5 / 2; |
+| DIV          |  Divide (returns an integer value if both operands are integers)        | SELECT VALUE 5 DIV 2; |
+| MOD (%)      |  Modulo                                                                 | SELECT VALUE 5 % 2; |
 | ^            |  Exponentiation                                                         | SELECT VALUE 2^3;       |
 | &#124;&#124; |  String concatenation                                                   | SELECT VALUE "ab"&#124;&#124;"c"&#124;&#124;"d";       |
 
@@ -103,16 +105,16 @@
 
 The following table enumerates all of SQL++'s comparison operators.
 
-| Operator       |  Purpose                                   | Example    |
-|----------------|--------------------------------------------|------------|
+| Operator       |  Purpose                                       | Example    |
+|----------------|------------------------------------------------|------------|
 | IS NULL        |  Test if a value is NULL                       | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NULL; |
 | IS NOT NULL    |  Test if a value is not NULL                   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT NULL; |
 | IS MISSING     |  Test if a value is MISSING                    | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS MISSING; |
 | IS NOT MISSING |  Test if a value is not MISSING                | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT MISSING;|
 | IS UNKNOWN     |  Test if a value is NULL or MISSING            | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS UNKNOWN; |
 | IS NOT UNKNOWN |  Test if a value is neither NULL nor MISSING   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT UNKNOWN;|
-| IS VALUED      |  Test if a value is neither NULL nor MISSING   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS VALUED; |
-| IS NOT VALUED  |  Test if a value is NULL or MISSING            | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT VALUED;|
+| IS KNOWN (IS VALUED) |  Test if a value is neither NULL nor MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS KNOWN; |
+| IS NOT KNOWN (IS NOT VALUED) |  Test if a value is NULL or MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT KNOWN; |
 | BETWEEN        |  Test if a value is between a start value and <br/>a end value. The comparison is inclusive <br/>to both start and end values. |  SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId BETWEEN 10 AND 20;|
 | =              |  Equality test                                 | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId=10; |
 | !=             |  Inequality test                               | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId!=10;|
@@ -134,8 +136,8 @@
 | IS NOT MISSING | TRUE | TRUE | FALSE |
 | IS UNKNOWN | FALSE | TRUE | TRUE |
 | IS NOT UNKNOWN | TRUE | FALSE | FALSE|
-| IS VALUED | TRUE | FALSE | FALSE |
-| IS NOT VALUED | FALSE | TRUE | TRUE |
+| IS KNOWN (IS VALUED) | TRUE | FALSE | FALSE |
+| IS NOT KNOWN (IS NOT VALUED) | FALSE | TRUE | TRUE |
 
 ### <a id="Logical_operators">Logical Operators</a>
 Logical operators perform logical `NOT`, `AND`, and `OR` operations over Boolean values (`TRUE` and `FALSE`) plus `NULL` and `MISSING`.
@@ -170,20 +172,6 @@
 | NULL | NULL |
 | MISSING | MISSING |
 
-## <a id="Case_expressions">Case Expressions</a>
-
-    CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
-    SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
-    SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
-
-In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
-
-In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
-
-The following example illustrates the form of a case expression.
-##### Example
-
-    CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
 
 ## <a id="Quantified_expressions">Quantified Expressions</a>
 
@@ -241,11 +229,13 @@
                   | VariableReference
                   | ParenthesizedExpression
                   | FunctionCallExpression
+                  | CaseExpression
                   | Constructor
 
 The most basic building block for any SQL++ expression is PrimaryExpression. This can be a simple literal (constant)
-value, a reference to a query variable that is in scope, a parenthesized expression, a function call, or a newly
-constructed instance of the data model (such as a newly constructed object, array, or multiset of data model instances).
+value, a reference to a query variable that is in scope, a parenthesized expression, a function call, a conditional 
+(case) expression, or a newly constructed instance of the data model (such as a newly constructed object, array, 
+or multiset of data model instances).
 
 ## <a id="Literals">Literals</a>
 
@@ -361,6 +351,22 @@
 
     length('a string')
 
+## <a id="Case_expressions">Case Expressions</a>
+
+    CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
+    SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+    SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+
+In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
+
+In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
+
+The following example illustrates the form of a case expression.
+
+##### Example
+
+    CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
+
 
 ### <a id="Constructors">Constructors</a>
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
index 996bb9e..223e188 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -1080,11 +1080,11 @@
 
 | Function       | NULL         | MISSING      | Empty Collection |
 |----------------|--------------|--------------|------------------|
-| COLL_COUNT     | counted      | counted      | 0                |
-| COLL_SUM       | returns NULL | returns NULL | returns NULL     |
-| COLL_MAX       | returns NULL | returns NULL | returns NULL     |
-| COLL_MIN       | returns NULL | returns NULL | returns NULL     |
-| COLL_AVG       | returns NULL | returns NULL | returns NULL     |
+| STRICT_COUNT   | counted      | counted      | 0                |
+| STRICT_SUM     | returns NULL | returns NULL | returns NULL     |
+| STRICT_MAX     | returns NULL | returns NULL | returns NULL     |
+| STRICT_MIN     | returns NULL | returns NULL | returns NULL     |
+| STRICT_AVG     | returns NULL | returns NULL | returns NULL     |
 | ARRAY_COUNT    | not counted  | not counted  | 0                |
 | ARRAY_SUM      | ignores NULL | ignores NULL | returns NULL     |
 | ARRAY_MAX      | ignores NULL | ignores NULL | returns NULL     |
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
index bc0b398..1328ed9 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
@@ -44,73 +44,61 @@
         ftcontains(Expression1, Expression2, {FullTextOption})
         ftcontains(Expression1, Expression2)
 
-For example, we can execute the following query to find tweet messages where the `message-text` field includes
+For example, we can execute the following query to find Chirp messages where the `messageText` field includes
 “voice” as a word. Please note that an FTS search is case-insensitive.
 Thus, "Voice" or "voice" will be evaluated as the same word.
 
-        use dataverse TinySocial;
-
-        for $msg in dataset TweetMessages
-        where ftcontains($msg.message-text, "voice", {"mode":"any"})
-        return {"id": $msg.id}
-
-The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
-
-The same query can be also expressed in the SQL++.
-
         use TinySocial;
 
-        select element {"id":msg.id}
-        from TweetMessages as msg
-        where TinySocial.ftcontains(msg.`message-text`, "voice", {"mode":"any"})
+        select element {"chirpId": msg.chirpId}
+        from ChirpMessages msg
+        where ftcontains(msg.messageText, "voice", {"mode":"any"});
+
+The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
 
 The `Expression1` is an expression that should be evaluable as a string at runtime as in the above example
-where `$msg.message-text` is a string field. The `Expression2` can be a string, an (un)ordered list
+where `msg.messageText` is a string field. The `Expression2` can be a string, an (un)ordered list
 of string value(s), or an expression. In the last case, the given expression should be evaluable
 into one of the first two types, i.e., into a string value or an (un)ordered list of string value(s).
 
 The following examples are all valid expressions.
 
-       ... where ftcontains($msg.message-text, "sound")
-       ... where ftcontains($msg.message-text, "sound", {"mode":"any"})
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
-       ... where ftcontains($msg.message-text, {{"speed", "stand", "customization"}}, {"mode":"all"})
-       ... where ftcontains($msg.message-text, let $keyword_list := ["voice", "system"] return $keyword_list, {"mode":"all"})
-       ... where ftcontains($msg.message-text, $keyword_list, {"mode":"any"})
-
-In the last example above, `$keyword_list` should evaluate to a string or an (un)ordered list of string value(s).
+       ... where ftcontains(msg.messageText, "sound")
+       ... where ftcontains(msg.messageText, "sound", {"mode":"any"})
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
+       ... where ftcontains(msg.messageText, {{"speed", "stand", "customization"}}, {"mode":"all"})
 
 The last `FullTextOption` parameter clarifies the given FTS request. If you omit the `FullTextOption` parameter,
 then the default value will be set for each possible option. Currently, we only have one option named `mode`.
 And as we extend the FTS feature, more options will be added. Please note that the format of `FullTextOption`
 is a record, thus you need to put the option(s) in a record `{}`.
 The `mode` option indicates whether the given FTS query is a conjunctive (AND) or disjunctive (OR) search request.
-This option can be either `“any”` or `“all”`. The default value for `mode` is `“all”`. If one specifies `“any”`,
-a disjunctive search will be conducted. For example, the following query will find documents whose `message-text`
+This option can be either `“all”` (AND) or `“any”` (OR). The default value for `mode` is `“all”`. If one specifies `“any”`,
+a disjunctive search will be conducted. For example, the following query will find documents whose `messageText`
 field contains “sound” or “system”, so a document will be returned if it contains either “sound”, “system”,
 or both of the keywords.
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
 
 The other option parameter,`“all”`, specifies a conjunctive search. The following examples will find the documents whose
-`message-text` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
+`messageText` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
 not both, it will not be returned.
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
-       ... where ftcontains($msg.message-text, ["sound", "system"])
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+       ... where ftcontains(msg.messageText, ["sound", "system"])
 
 Currently AsterixDB doesn’t (yet) support phrase searches, so the following query will not work.
 
-       ... where ftcontains($msg.message-text, "sound system", {"mode":"any"})
+       ... where ftcontains(msg.messageText, "sound system", {"mode":"any"})
 
 As a workaround solution, the following query can be used to achieve a roughly similar goal. The difference is that
-the following queries will find documents where `$msg.message-text` contains both “sound” and “system”, but the order
+the following queries will find documents where `msg.messageText` contains both “sound” and “system”, but the order
 and adjacency of “sound” and “system” are not checked, unlike in a phrase search. As a result, the query below would
 also return documents with “sound system can be installed.”, “system sound is perfect.”,
 or “sound is not clear. You may need to install a new system.”
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
-       ... where ftcontains($msg.message-text, ["sound", "system"])
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+       ... where ftcontains(msg.messageText, ["sound", "system"])
 
 
 ## <a id="FulltextIndex">Creating and utilizing a Full-text index</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
@@ -118,6 +106,9 @@
 When there is a full-text index on the field that is being searched, rather than scanning all records,
 AsterixDB can utilize that index to expedite the execution of a FTS query. To create a full-text index,
 you need to specify the index type as `fulltext` in your DDL statement. For instance, the following DDL
-statement create a full-text index on the TweetMessages.message-text attribute.
+statement create a full-text index on the `GleambookMessages.message` attribute. Note that a full-text index
+cannot be built on a dataset with the variable-length primary key (e.g., string).
 
-    create index messageFTSIdx on TweetMessages(message-text) type fulltext;
+    use TinySocial;
+
+    create index messageFTSIdx on GleambookMessages(message) type fulltext;
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
index 95c752f..43a7cd5 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
@@ -283,7 +283,7 @@
 ### Arithmetic Expressions
 
     AddExpr  ::= MultExpr ( ( "+" | "-" ) MultExpr )*
-    MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | "^"| "idiv" ) UnaryExpr )*
+    MultExpr ::= UnaryExpr ( ( "*" | "/" | "div" | "%" | "mod" | "^" ) UnaryExpr )*
     UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
 
 AQL also supports the usual cast of characters for arithmetic expressions.
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
index 0d949db..8118126 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
@@ -45,8 +45,8 @@
 AsterixDB supports [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and
 [Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets).  For
 instance, in our
-[TinySocial](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
-example, the `friend-ids` of a Facebook user forms a set
+[TinySocial](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
+example, the `friendIds` of a Gleambook user forms a set
 of friends, and we can define a similarity between the sets of
 friends of two users. We can also convert a string to a set of grams of a length "n"
 (called "n-grams") and define the Jaccard similarity between the two
@@ -55,50 +55,45 @@
 `schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
 
 AsterixDB provides
-[tokenization functions](functions.html#Tokenizing_Functions)
+[tokenization functions](../sqlpp/builtins.html#Tokenizing_Functions)
 to convert strings to sets, and the
-[similarity functions](functions.html#Similarity_Functions).
+[similarity functions](../sqlpp/builtins.html#Similarity_Functions).
 
 ## <a id="SimilaritySelectionQueries">Similarity Selection Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
 The following query
-asks for all the Facebook users whose name is similar to
+asks for all the Gleambook users whose name is similar to
 `Suzanna Tilson`, i.e., their edit distance is at most 2.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance($user.name, "Suzanna Tilson")
-        where $ed <= 2
-        return $user
-
+        select u
+        from GleambookUsers u
+        where edit_distance(u.name, "Suzanna Tilson") <= 2;
 
 The following query
-asks for all the Facebook users whose set of friend ids is
+asks for all the Gleambook users whose set of friend ids is
 similar to `[1,5,9,10]`, i.e., their Jaccard similarity is at least 0.6.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
-        where $sim >= 0.6f
-        return $user
-
+        select u
+        from GleambookUsers u
+        where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
 
 AsterixDB allows a user to use a similarity operator `~=` to express a
 condition by defining the similarity function and threshold
 using "set" statements earlier. For instance, the above query can be
 equivalently written as:
 
-        use dataverse TinySocial;
+        use TinySocial;
 
         set simfunction "jaccard";
         set simthreshold "0.6f";
 
-        for $user in dataset('FacebookUsers')
-        where $user.friend-ids ~= [1,5,9,10]
-        return $user
-
+        select u
+        from GleambookUsers u
+        where u.friendIds ~= [1,5,9,10];
 
 In this query, we first declare Jaccard as the similarity function
 using `simfunction` and then specify the threshold `0.6f` using
@@ -107,27 +102,19 @@
 ## <a id="SimilarityJoinQueries">Similarity Join Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
 AsterixDB supports fuzzy joins between two sets. The following
-[query](primer.html#Query_5_-_Fuzzy_Join)
-finds, for each Facebook user, all Twitter users with names
+[query](../sqlpp/primer-sqlpp.html#Query_5_-_Fuzzy_Join)
+finds, for each Gleambook user, all Chirp users with names
 similar to their name based on the edit distance.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
         set simfunction "edit-distance";
         set simthreshold "3";
 
-        for $fbu in dataset FacebookUsers
-        return {
-            "id": $fbu.id,
-            "name": $fbu.name,
-            "similar-users": for $t in dataset TweetMessages
-                                let $tu := $t.user
-                                where $tu.name ~= $fbu.name
-                                return {
-                                "twitter-screenname": $tu.screen-name,
-                                "twitter-name": $tu.name
-                                }
-        };
+        select gbu.id, gbu.name, (select cu.screenName, cu.name
+                                  from ChirpUsers cu
+                                  where cu.name ~= gbu.name) as similar_users
+        from GleambookUsers gbu;
 
 ## <a id="UsingIndexesToSupportSimilarityQueries">Using Indexes to Support Similarity Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
@@ -146,101 +133,95 @@
 [paper](http://www.ics.uci.edu/~chenli/pub/icde2009-memreducer.pdf).
 
 For instance, the following DDL statements create an ngram index on the
-`FacebookUsers.name` attribute using an inverted index of 3-grams.
+`GleambookUsers.name` attribute using an inverted index of 3-grams.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        create index fbUserIdx on FacebookUsers(name) type ngram(3);
+        create index gbUserIdx on GleambookUsers(name) type ngram(3);
 
 The number "3" in "ngram(3)" is the length "n" in the grams. This
 index can be used to optimize similarity queries on this attribute
 using
-[edit-distance](functions.html#edit-distance),
-[edit-distance-check](functions.html#edit-distance-check),
-[similarity-jaccard](functions.html#similarity-jaccard),
-or [similarity-jaccard-check](functions.html#similarity-jaccard-check)
+[edit_distance](../sqlpp/builtins.html#edit_distance),
+[edit_distance_check](../sqlpp/builtins.html#edit_distance_check),
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard),
+or [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check)
 queries on this attribute where the
 similarity is defined on sets of 3-grams.  This index can also be used
-to optimize queries with the "[contains()]((functions.html#contains))" predicate (i.e., substring
+to optimize queries with the "[contains()]((../sqlpp/builtins.html#contains))" predicate (i.e., substring
 matching) since it can be also be solved by counting on the inverted
 lists of the grams in the query string.
 
-#### NGram Index usage case - [edit-distance](functions.html#edit-distance) ####
+#### NGram Index usage case - [edit_distance](../sqlpp/builtins.html#edit-distance) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance($user.name, "Suzanna Tilson")
-        where $ed <= 2
-        return $user
+        select u
+        from GleambookUsers u
+        where edit_distance(u.name, "Suzanna Tilson") <= 2;
 
-#### NGram Index usage case - [edit-distance-check](functions.html#edit-distance-check) ####
+#### NGram Index usage case - [edit_distance_check](../sqlpp/builtins.html#edit_distance_check) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance-check($user.name, "Suzanna Tilson", 2)
-        where $ed[0]
-        return $ed[1]
+        select u
+        from GleambookUsers u
+        where edit_distance_check(u.name, "Suzanna Tilson", 2)[0];
 
-#### NGram Index usage case - [similarity-jaccard](functions.html#similarity-jaccard) ####
+#### NGram Index usage case - [contains()]((../sqlpp/builtins.html#contains)) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
-        where $sim >= 0.6f
-        return $user
-
-#### NGram Index usage case - [similarity-jaccard-check](functions.html#similarity-jaccard-check) ####
-
-        use dataverse TinySocial;
-
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard-check($user.friend-ids, [1,5,9,10], 0.6f)
-        where $sim[0]
-        return $user
-
-#### NGram Index usage case - [contains()]((functions.html#contains)) ####
-
-        use dataverse TinySocial;
-
-        for $i in dataset('FacebookMessages')
-        where contains($i.message, "phone")
-        return {"mid": $i.message-id, "message": $i.message}
+        select m
+        from GleambookMessages m
+        where contains(m.message, "phone");
 
 
 ### Keyword Index ###
 
-A "keyword index" is constructed on a set of strings or sets (e.g., OrderedList, UnorderedList). Instead of
+A "keyword index" is constructed on a set of strings or sets (e.g., array, multiset). Instead of
 generating grams as in an ngram index, we generate tokens (e.g., words) and for each token, construct an inverted list that includes the ids of the
 objects with this token.  The following two examples show how to create keyword index on two different types:
 
 
 #### Keyword Index on String Type ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        drop index FacebookMessages.fbMessageIdx if exists;
-        create index fbMessageIdx on FacebookMessages(message) type keyword;
+        drop index GleambookMessages.gbMessageIdx if exists;
+        create index gbMessageIdx on GleambookMessages(message) type keyword;
 
-        for $o in dataset('FacebookMessages')
-        let $jacc := similarity-jaccard-check(word-tokens($o.message), word-tokens("love like ccast"), 0.2f)
-        where $jacc[0]
-        return $o
+        select m
+        from GleambookMessages m
+        where similarity_jaccard_check(word_tokens(m.message), word_tokens("love like ccast"), 0.2f)[0];
 
-#### Keyword Index on UnorderedList Type ####
+#### Keyword Index on Multiset Type ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        create index fbUserIdx_fids on FacebookUsers(friend-ids) type keyword;
+        create index gbUserIdxFIds on GleambookUsers(friendIds) type keyword;
 
-        for $c in dataset('FacebookUsers')
-        let $jacc := similarity-jaccard-check($c.friend-ids, {{3,10}}, 0.5f)
-        where $jacc[0]
-        return $c
+        select u
+        from GleambookUsers u
+        where similarity_jaccard_check(u.friendIds, {{3,10}}, 0.5f)[0];
 
 As shown above, keyword index can be used to optimize queries with token-based similarity predicates, including
-[similarity-jaccard](functions.html#similarity-jaccard) and
-[similarity-jaccard-check](functions.html#similarity-jaccard-check).
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) and
+[similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check).
+
+#### Keyword Index usage case - [similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) ####
+
+        use TinySocial;
+
+        select u
+        from GleambookUsers u
+        where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
+
+#### Keyword Index usage case - [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check) ####
+
+        use TinySocial;
+
+        select u
+        from GleambookUsers u
+        where similarity_jaccard_check(u.friendIds, [1,5,9,10], 0.6f)[0];
 
diff --git a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
index d2d4488..f5635b8 100644
--- a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
+++ b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
@@ -23,7 +23,7 @@
 
 * [Introduction](#Introduction)
 * [Feed Adapters](#FeedAdapters)
-<!-- * [Feed Policies](#FeedPolicies) -->
+* [Feed Policies](#FeedPolicies)
 
 ## <a name="Introduction">Introduction</a>  ##
 
@@ -61,11 +61,13 @@
 ####Ingesting Twitter Stream
 We shall use the built-in push-based Twitter adapter.
 As a pre-requisite, we must define a Tweet using the AsterixDB Data Model (ADM)
-and the AsterixDB Query Language (AQL). Given below are the type definitions in AQL
+and the query language SQL++. Given below are the type definitions in SQL++
 that create a Tweet datatype which is representative of a real tweet as obtained from Twitter.
 
+        drop dataverse feeds if exists;
+
         create dataverse feeds;
-        use dataverse feeds;
+        use feeds;
 
         create type TwitterUser as closed {
             screen_name: string,
@@ -77,13 +79,12 @@
         create type Tweet as open {
             id: int64,
             user: TwitterUser
-        }
+        };
 
-        create dataset Tweets (Tweet)
-        primary key id;
+        create dataset Tweets (Tweet) primary key id;
 
 We also create a dataset that we shall use to persist the tweets in AsterixDB.
-Next we make use of the `create feed` AQL statement to define our example data feed.
+Next we make use of the `create feed` SQL++ statement to define our example data feed.
 
 #####Using the "push_twitter" feed adapter#####
 The "push_twitter" adapter requires setting up an application account with Twitter. To retrieve
@@ -91,6 +92,7 @@
 a name and a brief description for the application. Each application has associated OAuth
 authentication credentials that include OAuth keys and tokens. Accessing the
 Twitter API requires providing the following.
+
 1. Consumer Key (API Key)
 2. Consumer Secret (API Secret)
 3. Access Token
@@ -101,18 +103,20 @@
 using the "push_twitter" adapter. For further information on obtaining OAuth keys and tokens and
 registering an application with Twitter, please visit http://apps.twitter.com
 
-Given below is an example AQL statement that creates a feed called "TwitterFeed" by using the
+Given below is an example SQL++ statement that creates a feed called "TwitterFeed" by using the
 "push_twitter" adapter.
 
-        use dataverse feeds;
+        use feeds;
 
-        create feed TwitterFeed if not exists using "push_twitter"
-        (("type-name"="Tweet"),
-         ("format"="twitter-status"),
-         ("consumer.key"="************"),
-         ("consumer.secret"="**************"),
-         ("access.token"="**********"),
-         ("access.token.secret"="*************"));
+        create feed TwitterFeed with {
+          "adapter-name": "push_twitter",
+          "type-name": "Tweet",
+          "format": "twitter-status",
+          "consumer.key": "************",
+          "consumer.secret": "************",
+          "access.token": "**********",
+          "access.token.secret": "*************"
+        };
 
 It is required that the above authentication parameters are provided valid.
 Note that the `create feed` statement does not initiate the flow of data from Twitter into
@@ -122,23 +126,25 @@
 
 The Twitter adapter also supports several Twitter streaming APIs as follow:
 
-1. Track filter ("keywords"="AsterixDB, Apache")
-2. Locations filter ("locations"="-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358")
-3. Language filter ("language"="en")
-4. Filter level ("filter-level"="low")
+1. Track filter `"keywords": "AsterixDB, Apache"`
+2. Locations filter `"locations": "-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358"`
+3. Language filter `"language": "en"`
+4. Filter level `"filter-level": "low"`
 
 An example of Twitter adapter tracking tweets with keyword "news" can be described using following ddl:
 
-        use dataverse feeds;
+        use feeds;
 
-        create feed TwitterFeed if not exists using "push_twitter"
-        (("type-name"="Tweet"),
-         ("format"="twitter-status"),
-         ("consumer.key"="************"),
-         ("consumer.secret"="**************"),
-         ("access.token"="**********"),
-         ("access.token.secret"="*************"),
-         ("keywords"="news"));
+        create feed TwitterFeed with {
+          "adapter-name": "push_twitter",
+          "type-name": "Tweet",
+          "format": "twitter-status",
+          "consumer.key": "************",
+          "consumer.secret": "************",
+          "access.token": "**********",
+          "access.token.secret": "*************",
+          "keywords": "news"
+        };
 
 For more details about these APIs, please visit https://dev.twitter.com/streaming/overview/request-parameters
 
@@ -154,7 +160,7 @@
 contents of the dataset represent the union of the connected feeds.
 Also one feed can be simultaneously connected to multiple target datasets.
 
-        use dataverse feeds;
+        use feeds;
 
         connect feed TwitterFeed to dataset Tweets;
 
@@ -170,108 +176,53 @@
 Let the feed run for a minute, then run the following query to see the
 latest tweets that are stored into the data set.
 
-        use dataverse feeds;
+        use feeds;
 
-        for $i in dataset Tweets limit 10 return $i;
+        select * from Tweets limit 10;
 
 The dataflow of data from a feed can be terminated explicitly by `stop feed` statement.
 
-        use dataverse feeds;
+        use feeds;
 
         stop feed TwitterFeed;
 
 The `disconnnect statement` can be used to disconnect the feed from certain dataset.
 
-        use dataverse feeds;
+        use feeds;
 
         disconnect feed TwitterFeed from dataset Tweets;
 
 ###Ingesting with Other Adapters
 AsterixDB has several builtin feed adapters for data ingestion. User can also
 implement their own adapters and plug them into AsterixDB.
-Here we introduce `rss_feed`, `socket_adapter` and `localfs`
+Here we introduce `socket_adapter` and `localfs`
 feed adapter that cover most of the common application scenarios.
 
-#####Using the "rss_feed" feed adapter#####
-`rss_feed` adapter allows retrieving data given a collection of RSS end point URLs.
-As observed in the case of ingesting tweets, it is required to model an RSS data item using AQL.
-
-        use dataverse feeds;
-
-        create type Rss if not exists as open {
-            id: string,
-            title: string,
-            description: string,
-            link: string
-        };
-
-        create dataset RssDataset (Rss)
-        primary key id;
-
-Next, we define an RSS feed using our built-in adapter "rss_feed".
-
-        use dataverse feeds;
-
-        create feed my_feed using
-        rss_feed (
-           ("type-name"="Rss"),
-           ("format"="rss"),
-           ("url"="http://rss.cnn.com/rss/edition.rss")
-        );
-
-In the above definition, the configuration parameter "url" can be a comma-separated list that reflects a
-collection of RSS URLs, where each URL corresponds to an RSS endpoint or an RSS feed.
-The "rss_feed" retrieves data from each of the specified RSS URLs (comma separated values) in parallel.
-
-The following statements connect the feed into the `RssDataset`:
-
-        use dataverse feeds;
-
-        connect feed my_feed to dataset RssDataset;
-
-The following statements activate the feed and start the dataflow:
-
-        use dataverse feeds;
-
-        start feed my_feed;
-
-The following statements show the latest data from the data set, stop the feed, and
-disconnect the feed from the data set.
-
-        use dataverse feeds;
-
-        for $i in dataset RssDataset limit 10 return $i;
-
-        stop feed my_feed
-
-        disconnect feed my_feed from dataset RssDataset;
-
-
 #####Using the "socket_adapter" feed adapter#####
 `socket_adapter` feed opens a web socket on the given node which allows user to push data into
 AsterixDB directly. Here is an example:
 
         drop dataverse feeds if exists;
         create dataverse feeds;
-        use dataverse feeds;
+        use feeds;
 
         create type TestDataType as open {
            screenName: string
-        }
+        };
 
         create dataset TestDataset(TestDataType) primary key screenName;
 
-        create feed TestSocketFeed using socket_adapter
-        (
-           ("sockets"="127.0.0.1:10001"),
-           ("address-type"="IP"),
-           ("type-name"="TestDataType"),
-           ("format"="adm")
-        );
+        create feed TestSocketFeed with {
+          "adapter-name": "socket_adapter",
+          "sockets": "127.0.0.1:10001",
+          "address-type": "IP",
+          "type-name": "TestDataType",
+          "format": "adm"
+        };
 
         connect feed TestSocketFeed to dataset TestDataset;
 
-        use dataverse feeds;
+        use feeds;
         start feed TestSocketFeed;
 
 The above statements create a socket feed which is listening to "10001" port of the host machine. This feed accepts data
@@ -297,25 +248,27 @@
 `localfs` adapter enables data ingestion from local file system. It allows user to feed data records on local disk
 into a dataset. A DDL example for creating a `localfs` feed is given as follow:
 
-        use dataverse feeds;
+        use feeds;
 
-        create type TweetType as closed {
-          id: string,
-          username : string,
-          location : string,
-          text : string,
-          timestamp : string
-        }
+        create type TestDataType as open {
+           screenName: string
+        };
 
-        create dataset Tweets(TweetType)
-        primary key id;
+        create dataset TestDataset(TestDataType) primary key screenName;
 
-        create feed TweetFeed
-        using localfs
-        (("type-name"="TweetType"),("path"="HOSTNAME://LOCAL_FILE_PATH"),("format"="adm"))
+        create feed TestFileFeed with {
+          "adapter-name": "localfs",
+          "type-name": "TestDataType",
+          "path": "HOSTNAME://LOCAL_FILE_PATH",
+          "format": "adm"
+        };
+
+        connect feed TestFileFeed to dataset TestDataset;
+
+        start feed TestFileFeed;
 
 Similar to previous examples, we need to define the datatype and dataset this feed uses.
-The "path" parameter refers to the local datafile that we want to ingest data from.
+The "path" parameter refers to the local data file that we want to ingest data from.
 `HOSTNAME` can either be the IP address or node name of the machine which holds the file.
 `LOCAL_FILE_PATH` indicates the absolute path to the file on that machine. Similarly to `socket_adapter`,
 this feed takes `adm` formatted data records.
@@ -334,7 +287,7 @@
 Use that same datatype in feed definition will cause a type discrepancy since there is no such field in the datasource.
 Thus, we will need to define two separate datatypes for feed and dataset:
 
-        use dataverse feeds;
+        use feeds;
 
         create type DBLPFeedType as closed {
           dblpid: string,
@@ -352,13 +305,13 @@
         }
         create dataset DBLPDataset(DBLPDataSetType) primary key id autogenerated;
 
-        create feed DBLPFeed using socket_adapter
-        (
-            ("sockets"="127.0.0.1:10001"),
-            ("address-type"="IP"),
-            ("type-name"="DBLPFeedType"),
-            ("format"="adm")
-        );
+        create feed DBLPFeed with {
+          "adapter-name": "socket_adapter",
+          "sockets": "127.0.0.1:10001",
+          "address-type": "IP",
+          "type-name": "DBLPFeedType",
+          "format": "adm"
+        };
 
         connect feed DBLPFeed to dataset DBLPDataset;
 
@@ -403,7 +356,6 @@
 ingestion policy is specified as part of the `connect feed` statement
 or else the "Basic" policy will be chosen as the default.
 
-        use dataverse feeds;
+        use feeds;
 
-        connect feed TwitterFeed to dataset Tweets
-        using policy Basic;
\ No newline at end of file
+        connect feed TwitterFeed to dataset Tweets using policy Basic;
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/ncservice.md b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
index 605ee30..2b309ce 100644
--- a/asterixdb/asterix-doc/src/site/markdown/ncservice.md
+++ b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
@@ -366,7 +366,7 @@
 | common  | txn.lock.timeout.sweepthreshold           | Interval (in milliseconds) for checking lock timeout | 10000 |
 | common  | txn.lock.timeout.waitthreshold            | Time out (in milliseconds) of waiting for a lock | 60000 |
 | common  | txn.log.buffer.numpages                   | The number of pages in the transaction log tail | 8 |
-| common  | txn.log.buffer.pagesize                   | The page size (in bytes) for transaction log buffer | 131072 (128 kB) |
+| common  | txn.log.buffer.pagesize                   | The page size (in bytes) for transaction log buffer | 4194304 (4MB) |
 | common  | txn.log.checkpoint.history                | The number of checkpoints to keep in the transaction log | 0 |
 | common  | txn.log.checkpoint.lsnthreshold           | The checkpoint threshold (in terms of LSNs (log sequence numbers) that have been written to the transaction log, i.e., the length of the transaction log) for transaction logs | 67108864 (64 MB) |
 | common  | txn.log.checkpoint.pollfrequency          | The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be written | 120 |
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 78549b3..cbaa889 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -397,5 +397,10 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+	<dependency>
+	  <groupId>com.esri.geometry</groupId>
+	  <artifactId>esri-geometry-api</artifactId>
+	  <version>2.0.0</version>
+	</dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
index 08ffe18..9033814 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
@@ -77,4 +77,11 @@
     public default IFeedMarker getProgressReporter() {
         return null;
     }
+
+    /**
+     * @return JSON String containing ingestion stats
+     */
+    default String getStats() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
deleted file mode 100644
index 23c5bdd..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.api;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public interface IRecordWithMetaDataAndPKParser<T> extends IRecordDataParser<RecordWithMetadataAndPK<T>> {
-
-    public void parseMeta(RecordWithMetadataAndPK<? extends T> record, DataOutput out) throws IOException;
-
-    public void appendKeys(RecordWithMetadataAndPK<T> record, ArrayTupleBuilder tb) throws IOException;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
deleted file mode 100755
index 9f14ec0..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.api;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.base.AOrderedList;
-import org.apache.asterix.om.base.ARecord;
-import org.apache.asterix.om.base.IAObject;
-
-public interface IResultCollector {
-
-    public void writeIntResult(int result) throws AsterixException;
-
-    public void writeFloatResult(float result) throws AsterixException;
-
-    public void writeDoubleResult(double result) throws AsterixException;
-
-    public void writeStringResult(String result) throws AsterixException;
-
-    public void writeRecordResult(ARecord result) throws AsterixException;
-
-    public void writeListResult(AOrderedList list) throws AsterixException;
-
-    public IAObject getComplexTypeResultHolder();
-
-    public DataOutput getDataOutput();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 164ff68..f392139 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -40,6 +40,7 @@
 public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowController {
     public static final String INCOMING_RECORDS_COUNT_FIELD_NAME = "incoming-records-count";
     public static final String FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME = "failed-at-parser-records-count";
+    public static final String READER_STATS_FIELD_NAME = "reader-stats";
 
     public enum State {
         CREATED,
@@ -257,7 +258,15 @@
 
     @Override
     public String getStats() {
-        return "{\"" + INCOMING_RECORDS_COUNT_FIELD_NAME + "\": " + incomingRecordsCount + ", \""
-                + FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME + "\": " + failedRecordsCount + "}";
+        String readerStats = recordReader.getStats();
+        StringBuilder str = new StringBuilder();
+        str.append("{");
+        if (readerStats != null) {
+            str.append("\"").append(READER_STATS_FIELD_NAME).append("\":").append(readerStats).append(", ");
+        }
+        str.append("\"").append(INCOMING_RECORDS_COUNT_FIELD_NAME).append("\": ").append(incomingRecordsCount)
+                .append(", \"").append(FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME).append("\": ")
+                .append(failedRecordsCount).append("}");
+        return str.toString();
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
index edac0fa..f22693a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
@@ -33,8 +33,6 @@
 
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hyracks.api.client.HyracksConnection;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -53,19 +51,14 @@
     /** a map from the NC name to the index */
     private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
 
-    /** a map from NC name to the NodeControllerInfo */
-    private Map<String, NodeControllerInfo> ncNameToNcInfos;
-
     /**
      * The constructor of the scheduler.
      *
      * @param ncNameToNcInfos
      * @throws HyracksException
      */
-    public IndexingScheduler(String ipAddress, int port) throws HyracksException {
+    public IndexingScheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
         try {
-            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
-            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
             loadIPAddressToNCMap(ncNameToNcInfos);
         } catch (Exception e) {
             throw HyracksException.create(e);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
index 79dc396..df4c093 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
@@ -18,8 +18,11 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -36,7 +39,8 @@
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 
 /**
@@ -74,10 +78,13 @@
                 // Open the index
                 indexHelper.open();
                 try {
-                    IIndex index = indexHelper.getIndexInstance();
+                    ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
+                    Map<String, Object> parameters = new HashMap<>();
+                    parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+                            LSMComponentId.DEFAULT_COMPONENT_ID);
                     // Create bulk loader
                     IIndexBulkLoader bulkLoader =
-                            index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false);
+                            index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false, parameters);
                     // Load files
                     for (ExternalFile file : files) {
                         bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
index 4bc2867..ea99993 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
@@ -18,7 +18,9 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -69,9 +71,10 @@
                 indexHelper.open();
                 IIndex index = indexHelper.getIndexInstance();
                 LSMTwoPCBTreeBulkLoader bulkLoader = null;
+                Map<String, Object> parameters = new HashMap<>();
                 try {
                     bulkLoader = (LSMTwoPCBTreeBulkLoader) ((ExternalBTree) index)
-                            .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size());
+                            .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), parameters);
                     // Load files
                     // The files must be ordered according to their numbers
                     for (ExternalFile file : files) {
@@ -86,7 +89,7 @@
                             case NO_OP:
                                 break;
                             default:
-                                throw HyracksDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP,
+                                throw HyracksDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP, sourceLoc,
                                         file.getPendingOp());
                         }
                     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
index 573de5d..74bc0dc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
@@ -18,12 +18,18 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 
 public class ExternalIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
 
@@ -43,4 +49,12 @@
         super.open();
         ((ITwoPCIndex) index).setCurrentVersion(version);
     }
+
+    @Override
+    protected void initializeBulkLoader() throws HyracksDataException {
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
+        bulkLoader = ((ILSMIndex) index).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                parameters);
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
index aaca3f1..57e2917 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.external.operators;
 
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.om.base.AMutableInt32;
@@ -60,8 +62,9 @@
         try {
             writer.open();
             // Transactional BulkLoader
-            bulkLoader =
-                    ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length);
+            Map<String, Object> parameters = new HashMap<>();
+            bulkLoader = ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length,
+                    parameters);
             // Delete files
             for (int i = 0; i < deletedFiles.length; i++) {
                 fileNumber.setValue(deletedFiles[i]);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index c0ccf11..f1eba4c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.Serializable;
 import java.util.BitSet;
 import java.util.List;
 
@@ -52,6 +51,7 @@
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.asterix.om.util.container.ListObjectPool;
 import org.apache.asterix.runtime.operators.file.adm.AdmLexer;
+import org.apache.asterix.runtime.operators.file.adm.AdmLexer.TokenImage;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -72,64 +72,11 @@
     private final IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
             new ListObjectPool<IMutableValueStorage, ATypeTag>(new AbvsBuilderFactory());
 
+    private final TokenImage tmpTokenImage = new TokenImage();
+
     private final String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
     private final String mismatchErrorMessage2 = " got a value of type ";
 
-    static class ParseException extends HyracksDataException {
-        private static final long serialVersionUID = 1L;
-        private String filename;
-        private int line = -1;
-        private int column = -1;
-
-        public ParseException(String message) {
-            super(message);
-        }
-
-        public ParseException(int errorCode, Serializable... param) {
-            super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
-        }
-
-        public ParseException(int errorCode, Throwable e, Serializable... param) {
-            super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
-            addSuppressed(e);
-        }
-
-        public ParseException(Throwable cause) {
-            super(cause);
-        }
-
-        public ParseException(String message, Throwable cause) {
-            super(message, cause);
-        }
-
-        public ParseException(Throwable cause, String filename, int line, int column) {
-            super(cause);
-            setLocation(filename, line, column);
-        }
-
-        public void setLocation(String filename, int line, int column) {
-            this.filename = filename;
-            this.line = line;
-            this.column = column;
-        }
-
-        @Override
-        public String getMessage() {
-            StringBuilder msg = new StringBuilder("Parse error");
-            if (filename != null) {
-                msg.append(" in file " + filename);
-            }
-            if (line >= 0) {
-                if (column >= 0) {
-                    msg.append(" at (" + line + ", " + column + ")");
-                } else {
-                    msg.append(" in line " + line);
-                }
-            }
-            return msg.append(": " + super.getMessage()).toString();
-        }
-    }
-
     public ADMDataParser(ARecordType recordType, boolean isStream) {
         this(null, recordType, isStream);
     }
@@ -254,16 +201,26 @@
                 break;
             case AdmLexer.TOKEN_STRING_LITERAL:
                 if (checkType(ATypeTag.STRING, objectType)) {
-                    String tokenImage =
-                            admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    aString.setValue(admLexer.containsEscapes() ? replaceEscapes(tokenImage) : tokenImage);
-                    stringSerde.serialize(aString, out);
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    if (admLexer.containsEscapes()) {
+                        replaceEscapes(tmpTokenImage);
+                    }
+                    int begin = tmpTokenImage.getBegin() + 1;
+                    int len = tmpTokenImage.getLength() - 2;
+                    parseString(tmpTokenImage.getBuffer(), begin, len, out);
                 } else if (checkType(ATypeTag.UUID, objectType)) {
                     // Dealing with UUID type that is represented by a string
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    aUUID.parseUUIDString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                            tmpTokenImage.getLength() - 2);
+                    uuidSerde.serialize(aUUID, out);
+                } else if (checkType(ATypeTag.GEOMETRY, objectType)) {
+                    // Parse the string as a WKT-encoded geometry
                     String tokenImage =
                             admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    aUUID.parseUUIDString(tokenImage);
-                    uuidSerde.serialize(aUUID, out);
+                    aGeomtry.parseWKT(tokenImage);
+                    out.writeByte(ATypeTag.GEOMETRY.serialize());
+                    geomSerde.serialize(aGeomtry, out);
                 } else {
                     throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_TYPE_MISMATCH, objectType.getTypeName());
                 }
@@ -364,13 +321,14 @@
 
     }
 
-    private String replaceEscapes(String tokenImage) throws ParseException {
-        char[] chars = tokenImage.toCharArray();
-        int len = chars.length;
-        int readpos = 0;
-        int writepos = 0;
-        int movemarker = 0;
-        while (readpos < len) {
+    // TODO: This function should be optimized. Currently it has complexity of O(N*N)!
+    private void replaceEscapes(TokenImage tokenImage) throws ParseException {
+        char[] chars = tokenImage.getBuffer();
+        int end = tokenImage.getBegin() + tokenImage.getLength();
+        int readpos = tokenImage.getBegin();
+        int writepos = tokenImage.getBegin();
+        int movemarker = tokenImage.getBegin();
+        while (readpos < end) {
             if (chars[readpos] == '\\') {
                 moveChars(chars, movemarker, readpos, readpos - writepos);
                 switch (chars[readpos + 1]) {
@@ -407,8 +365,8 @@
             ++writepos;
             ++readpos;
         }
-        moveChars(chars, movemarker, len, readpos - writepos);
-        return new String(chars, 0, len - (readpos - writepos));
+        moveChars(chars, movemarker, end, readpos - writepos);
+        tokenImage.reset(chars, tokenImage.getBegin(), tokenImage.getLength() - (readpos - writepos));
     }
 
     private static void moveChars(char[] chars, int start, int end, int offset) {
@@ -508,16 +466,16 @@
                     expectingRecordField = false;
 
                     if (recType != null) {
-                        String fldName =
-                                admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
+                        admLexer.getLastTokenImage(tmpTokenImage);
+                        String fldName = new String(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                tmpTokenImage.getLength() - 2);
                         fieldId = recBuilder.getFieldId(fldName);
                         if ((fieldId < 0) && !recType.isOpen()) {
                             throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_EXTRA_FIELD_IN_CLOSED_RECORD,
                                     fldName);
                         } else if ((fieldId < 0) && recType.isOpen()) {
-                            aStringFieldName.setValue(admLexer.getLastTokenImage().substring(1,
-                                    admLexer.getLastTokenImage().length() - 1));
-                            stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+                            parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                    tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
                             openRecordField = true;
                             fieldType = null;
                         } else {
@@ -527,9 +485,9 @@
                             openRecordField = false;
                         }
                     } else {
-                        aStringFieldName.setValue(
-                                admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1));
-                        stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+                        admLexer.getLastTokenImage(tmpTokenImage);
+                        parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
                         openRecordField = true;
                         fieldType = null;
                     }
@@ -807,7 +765,13 @@
 
     private void parseToNumericTarget(ATypeTag typeTag, IAType objectType, DataOutput out) throws IOException {
         ATypeTag targetTypeTag = getTargetTypeTag(typeTag, objectType);
-        if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), targetTypeTag, out)) {
+        boolean parsed = false;
+        if (targetTypeTag != null) {
+            admLexer.getLastTokenImage(tmpTokenImage);
+            parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(),
+                    targetTypeTag, out);
+        }
+        if (!parsed) {
             throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
         }
     }
@@ -819,8 +783,13 @@
             castBuffer.reset();
             dataOutput = castBuffer.getDataOutput();
         }
-
-        if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), typeTag, dataOutput)) {
+        boolean parsed = false;
+        if (targetTypeTag != null) {
+            admLexer.getLastTokenImage(tmpTokenImage);
+            parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(), typeTag,
+                    dataOutput);
+        }
+        if (!parsed) {
             throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
         }
 
@@ -862,9 +831,11 @@
             if (token == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                 token = admLexer.next();
                 if (token == AdmLexer.TOKEN_STRING_LITERAL) {
-                    String unquoted =
-                            admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    if (!parseValue(unquoted, typeTag, dataOutput)) {
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    int begin = tmpTokenImage.getBegin() + 1;
+                    int len = tmpTokenImage.getLength() - 2;
+                    // unquoted value
+                    if (!parseValue(tmpTokenImage.getBuffer(), begin, len, typeTag, dataOutput)) {
                         throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_CONSTRUCTOR_MISSING_DESERIALIZER,
                                 AdmLexer.tokenKindToString(token));
                     }
@@ -890,85 +861,86 @@
                 objectType.getTypeName() + " got " + typeTag);
     }
 
-    private boolean parseValue(String unquoted, ATypeTag typeTag, DataOutput out) throws HyracksDataException {
+    private boolean parseValue(char[] buffer, int begin, int len, ATypeTag typeTag, DataOutput out)
+            throws HyracksDataException {
         switch (typeTag) {
             case BOOLEAN:
-                parseBoolean(unquoted, out);
+                parseBoolean(buffer, begin, len, out);
                 return true;
             case TINYINT:
-                parseInt8(unquoted, out);
+                parseInt8(buffer, begin, len, out);
                 return true;
             case SMALLINT:
-                parseInt16(unquoted, out);
+                parseInt16(buffer, begin, len, out);
                 return true;
             case INTEGER:
-                parseInt32(unquoted, out);
+                parseInt32(buffer, begin, len, out);
                 return true;
             case BIGINT:
-                parseInt64(unquoted, out);
+                parseInt64(buffer, begin, len, out);
                 return true;
             case FLOAT:
-                if ("INF".equals(unquoted)) {
+                if (matches("INF", buffer, begin, len)) {
                     aFloat.setValue(Float.POSITIVE_INFINITY);
-                } else if ("-INF".equals(unquoted)) {
+                } else if (matches("-INF", buffer, begin, len)) {
                     aFloat.setValue(Float.NEGATIVE_INFINITY);
                 } else {
-                    aFloat.setValue(Float.parseFloat(unquoted));
+                    aFloat.setValue(parseFloat(buffer, begin, len));
                 }
                 floatSerde.serialize(aFloat, out);
                 return true;
             case DOUBLE:
-                if ("INF".equals(unquoted)) {
+                if (matches("INF", buffer, begin, len)) {
                     aDouble.setValue(Double.POSITIVE_INFINITY);
-                } else if ("-INF".equals(unquoted)) {
+                } else if (matches("-INF", buffer, begin, len)) {
                     aDouble.setValue(Double.NEGATIVE_INFINITY);
                 } else {
-                    aDouble.setValue(Double.parseDouble(unquoted));
+                    aDouble.setValue(parseDouble(buffer, begin, len));
                 }
                 doubleSerde.serialize(aDouble, out);
                 return true;
             case STRING:
-                aString.setValue(unquoted);
-                stringSerde.serialize(aString, out);
+                parseString(buffer, begin, len, out);
                 return true;
             case TIME:
-                parseTime(unquoted, out);
+                parseTime(buffer, begin, len, out);
                 return true;
             case DATE:
-                parseDate(unquoted, out);
+                parseDate(buffer, begin, len, out);
                 return true;
             case DATETIME:
-                parseDateTime(unquoted, out);
+                parseDateTime(buffer, begin, len, out);
                 return true;
             case DURATION:
-                parseDuration(unquoted, out);
+                parseDuration(buffer, begin, len, out);
                 return true;
             case DAYTIMEDURATION:
-                parseDateTimeDuration(unquoted, out);
+                parseDateTimeDuration(buffer, begin, len, out);
                 return true;
             case YEARMONTHDURATION:
-                parseYearMonthDuration(unquoted, out);
+                parseYearMonthDuration(buffer, begin, len, out);
                 return true;
             case POINT:
-                parsePoint(unquoted, out);
+                parsePoint(buffer, begin, len, out);
                 return true;
             case POINT3D:
-                parse3DPoint(unquoted, out);
+                parse3DPoint(buffer, begin, len, out);
                 return true;
             case CIRCLE:
-                parseCircle(unquoted, out);
+                parseCircle(buffer, begin, len, out);
                 return true;
             case RECTANGLE:
-                parseRectangle(unquoted, out);
+                parseRectangle(buffer, begin, len, out);
                 return true;
             case LINE:
-                parseLine(unquoted, out);
+                parseLine(buffer, begin, len, out);
                 return true;
             case POLYGON:
-                APolygonSerializerDeserializer.parse(unquoted, out);
+                //TODO: optimize
+                APolygonSerializerDeserializer.parse(new String(buffer, begin, len), out);
                 return true;
             case UUID:
-                aUUID.parseUUIDString(unquoted);
+                aUUID.parseUUIDString(buffer, begin, len);
                 uuidSerde.serialize(aUUID, out);
                 return true;
             default:
@@ -976,39 +948,53 @@
         }
     }
 
-    private void parseBoolean(String bool, DataOutput out) throws HyracksDataException {
-        if (bool.equals("true")) {
+    private boolean matches(String value, char[] buffer, int begin, int len) {
+        if (len != value.length()) {
+            return false;
+        }
+        for (int i = 0; i < len; i++) {
+            if (value.charAt(i) != buffer[i + begin]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private void parseBoolean(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        if (matches("true", buffer, begin, len)) {
             booleanSerde.serialize(ABoolean.TRUE, out);
-        } else if (bool.equals("false")) {
+        } else if (matches("false", buffer, begin, len)) {
             booleanSerde.serialize(ABoolean.FALSE, out);
         } else {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "boolean");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "boolean");
         }
     }
 
-    private void parseInt8(String int8, DataOutput out) throws HyracksDataException {
+    private void parseInt8(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         byte value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int8.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int8.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int8.length(); offset++) {
-            if ((int8.charAt(offset) >= '0') && (int8.charAt(offset) <= '9')) {
-                value = (byte) (((value * 10) + int8.charAt(offset)) - '0');
-            } else if ((int8.charAt(offset) == 'i') && (int8.charAt(offset + 1) == '8')
-                    && ((offset + 2) == int8.length())) {
+        for (; offset < begin + len; offset++) {
+            if ((buffer[offset] >= '0') && (buffer[offset] <= '9')) {
+                value = (byte) (((value * 10) + buffer[offset]) - '0');
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '8' && offset + 2 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int8");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int8");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1017,29 +1003,31 @@
         int8Serde.serialize(aInt8, out);
     }
 
-    private void parseInt16(String int16, DataOutput out) throws HyracksDataException {
+    private void parseInt16(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         short value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int16.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int16.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int16.length(); offset++) {
-            if ((int16.charAt(offset) >= '0') && (int16.charAt(offset) <= '9')) {
-                value = (short) (((value * 10) + int16.charAt(offset)) - '0');
-            } else if ((int16.charAt(offset) == 'i') && (int16.charAt(offset + 1) == '1')
-                    && (int16.charAt(offset + 2) == '6') && ((offset + 3) == int16.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (short) ((value * 10) + buffer[offset] - '0');
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '1' && buffer[offset + 2] == '6'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int16");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int16");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1048,29 +1036,31 @@
         int16Serde.serialize(aInt16, out);
     }
 
-    private void parseInt32(String int32, DataOutput out) throws HyracksDataException {
+    private void parseInt32(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         int value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int32.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int32.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int32.length(); offset++) {
-            if ((int32.charAt(offset) >= '0') && (int32.charAt(offset) <= '9')) {
-                value = (((value * 10) + int32.charAt(offset)) - '0');
-            } else if ((int32.charAt(offset) == 'i') && (int32.charAt(offset + 1) == '3')
-                    && (int32.charAt(offset + 2) == '2') && ((offset + 3) == int32.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (value * 10) + buffer[offset] - '0';
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '3' && buffer[offset + 2] == '2'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int32");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int32");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1080,29 +1070,31 @@
         int32Serde.serialize(aInt32, out);
     }
 
-    private void parseInt64(String int64, DataOutput out) throws HyracksDataException {
+    private void parseInt64(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         long value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int64.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int64.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int64.length(); offset++) {
-            if ((int64.charAt(offset) >= '0') && (int64.charAt(offset) <= '9')) {
-                value = (((value * 10) + int64.charAt(offset)) - '0');
-            } else if ((int64.charAt(offset) == 'i') && (int64.charAt(offset + 1) == '6')
-                    && (int64.charAt(offset + 2) == '4') && ((offset + 3) == int64.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (value * 10) + buffer[offset] - '0';
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '6' && buffer[offset + 2] == '4'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int64");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int64");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1127,4 +1119,4 @@
         admLexer.reInit(new InputStreamReader(in));
         return true;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
index 8351931..859ac22 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
@@ -19,7 +19,10 @@
 package org.apache.asterix.external.parser;
 
 import java.io.DataOutput;
+import java.io.IOException;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.external.api.IDataParser;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABinary;
@@ -31,6 +34,7 @@
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.ADuration;
 import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
@@ -45,6 +49,7 @@
 import org.apache.asterix.om.base.AMutableDouble;
 import org.apache.asterix.om.base.AMutableDuration;
 import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableGeometry;
 import org.apache.asterix.om.base.AMutableInt16;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
@@ -71,11 +76,14 @@
 import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import org.apache.asterix.om.base.temporal.ATimeParserFactory;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.bytes.Base64Parser;
 import org.apache.hyracks.util.bytes.HexParser;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
 
 /**
  * Base class for data parsers. Includes the common set of definitions for
@@ -93,6 +101,7 @@
     protected AMutableBinary aBinary = new AMutableBinary(null, 0, 0);
     protected AMutableString aStringFieldName = new AMutableString("");
     protected AMutableUUID aUUID = new AMutableUUID();
+    protected AMutableGeometry aGeomtry = new AMutableGeometry(null);
     // For temporal and spatial data types
     protected AMutableTime aTime = new AMutableTime(0);
     protected AMutableDateTime aDateTime = new AMutableDateTime(0L);
@@ -140,6 +149,9 @@
     protected ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
+    protected final AStringSerializerDeserializer untaggedStringSerde =
+            new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
+
     protected final HexParser hexParser = new HexParser();
     protected final Base64Parser base64Parser = new Base64Parser();
 
@@ -150,6 +162,9 @@
     protected ISerializerDeserializer<AUUID> uuidSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AUUID);
 
+    protected ISerializerDeserializer<AGeometry> geomSerde =
+            SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+
     // To avoid race conditions, the serdes for temporal and spatial data types needs to be one per parser
     // ^^^^^^^^^^^^^^^^^^^^^^^^ ??? then why all these serdes are static?
     @SuppressWarnings("unchecked")
@@ -195,14 +210,14 @@
         this.filename = filename;
     }
 
-    protected void parseTime(String time, DataOutput out) throws HyracksDataException {
-        int chrononTimeInMs = ATimeParserFactory.parseTimePart(time, 0, time.length());
+    protected void parseTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        int chrononTimeInMs = ATimeParserFactory.parseTimePart(buffer, begin, len);
         aTime.setValue(chrononTimeInMs);
         timeSerde.serialize(aTime, out);
     }
 
-    protected void parseDate(String date, DataOutput out) throws HyracksDataException {
-        long chrononTimeInMs = ADateParserFactory.parseDatePart(date, 0, date.length());
+    protected void parseDate(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, len);
         short temp = 0;
         if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
             temp = 1;
@@ -211,84 +226,93 @@
         dateSerde.serialize(aDate, out);
     }
 
-    protected void parseDateTime(String datetime, DataOutput out) throws HyracksDataException {
+    protected void parseDateTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         // +1 if it is negative (-)
-        short timeOffset = (short) ((datetime.charAt(0) == '-') ? 1 : 0);
 
-        timeOffset += 8;
+        int timeOffset = (buffer[begin] == '-') ? 1 : 0;
 
-        if (datetime.charAt(timeOffset) != 'T') {
+        timeOffset = timeOffset + 8 + begin;
+
+        if (buffer[timeOffset] != 'T') {
             timeOffset += 2;
-            if (datetime.charAt(timeOffset) != 'T') {
-                throw new HyracksDataException("This can not be an instance of datetime: missing T");
+            if (buffer[timeOffset] != 'T') {
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_INTERVAL_INVALID_DATETIME);
             }
         }
-        long chrononTimeInMs = ADateParserFactory.parseDatePart(datetime, 0, timeOffset);
-        chrononTimeInMs +=
-                ATimeParserFactory.parseTimePart(datetime, timeOffset + 1, datetime.length() - timeOffset - 1);
+        long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, timeOffset - begin);
+        chrononTimeInMs += ATimeParserFactory.parseTimePart(buffer, timeOffset + 1, begin + len - timeOffset - 1);
         aDateTime.setValue(chrononTimeInMs);
         datetimeSerde.serialize(aDateTime, out);
     }
 
-    protected void parseDuration(String duration, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
+    protected void parseDuration(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aDuration, ADurationParseOption.All);
         durationSerde.serialize(aDuration, out);
     }
 
-    protected void parseDateTimeDuration(String durationString, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
-                ADurationParseOption.All);
+    protected void parseDateTimeDuration(char[] buffer, int begin, int len, DataOutput out)
+            throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aDayTimeDuration, ADurationParseOption.All);
         dayTimeDurationSerde.serialize(aDayTimeDuration, out);
     }
 
-    protected void parseYearMonthDuration(String durationString, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
-                ADurationParseOption.All);
+    protected void parseYearMonthDuration(char[] buffer, int begin, int len, DataOutput out)
+            throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aYearMonthDuration, ADurationParseOption.All);
         yearMonthDurationSerde.serialize(aYearMonthDuration, out);
     }
 
-    protected void parsePoint(String point, DataOutput out) throws HyracksDataException {
+    protected void parsePoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            aPoint.setValue(Double.parseDouble(point.substring(0, point.indexOf(','))),
-                    Double.parseDouble(point.substring(point.indexOf(',') + 1, point.length())));
+            int commaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, commaIndex - begin),
+                    parseDouble(buffer, commaIndex + 1, begin + len - commaIndex - 1));
             pointSerde.serialize(aPoint, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point + " can not be an instance of point");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "point");
         }
     }
 
-    protected void parse3DPoint(String point3d, DataOutput out) throws HyracksDataException {
+    protected void parse3DPoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            int firstCommaIndex = point3d.indexOf(',');
-            int secondCommaIndex = point3d.indexOf(',', firstCommaIndex + 1);
-            aPoint3D.setValue(Double.parseDouble(point3d.substring(0, firstCommaIndex)),
-                    Double.parseDouble(point3d.substring(firstCommaIndex + 1, secondCommaIndex)),
-                    Double.parseDouble(point3d.substring(secondCommaIndex + 1, point3d.length())));
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            int secondCommaIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ',');
+            aPoint3D.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, secondCommaIndex - firstCommaIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             point3DSerde.serialize(aPoint3D, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point3d + " can not be an instance of point3d");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "point3d");
         }
     }
 
-    protected void parseCircle(String circle, DataOutput out) throws HyracksDataException {
+    protected void parseCircle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] parts = circle.split(" ");
-            aPoint.setValue(Double.parseDouble(parts[0].split(",")[0]), Double.parseDouble(parts[0].split(",")[1]));
-            aCircle.setValue(aPoint, Double.parseDouble(parts[1].substring(0, parts[1].length())));
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            int spaceIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ' ');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+            aCircle.setValue(aPoint, parseDouble(buffer, spaceIndex + 1, begin + len - spaceIndex - 1));
             circleSerde.serialize(aCircle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(circle + " can not be an instance of circle");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "circle");
         }
     }
 
-    protected void parseRectangle(String rectangle, DataOutput out) throws HyracksDataException {
+    protected void parseRectangle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] points = rectangle.split(" ");
-            if (points.length != 2) {
-                throw new HyracksDataException("rectangle consists of only 2 points.");
-            }
-            aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
-            aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+            int spaceIndex = indexOf(buffer, begin, len, ' ');
+
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+
+            int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+            aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             if (aPoint.getX() > aPoint2.getX() && aPoint.getY() > aPoint2.getY()) {
                 aRectangle.setValue(aPoint2, aPoint);
             } else if (aPoint.getX() < aPoint2.getX() && aPoint.getY() < aPoint2.getY()) {
@@ -298,23 +322,26 @@
                         "Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
             }
             rectangleSerde.serialize(aRectangle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(rectangle + " can not be an instance of rectangle");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "rectangle");
         }
     }
 
-    protected void parseLine(String line, DataOutput out) throws HyracksDataException {
+    protected void parseLine(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] points = line.split(" ");
-            if (points.length != 2) {
-                throw new HyracksDataException("line consists of only 2 points.");
-            }
-            aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
-            aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+            int spaceIndex = indexOf(buffer, begin, len, ' ');
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+            int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+            aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             aLine.setValue(aPoint, aPoint2);
             lineSerde.serialize(aLine, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(line + " can not be an instance of line");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "line");
         }
     }
 
@@ -357,4 +384,34 @@
 
         return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
     }
-}
+
+    protected double parseDouble(char[] buffer, int begin, int len) {
+        // TODO: parse double directly from char[]
+        String str = new String(buffer, begin, len);
+        return Double.valueOf(str);
+    }
+
+    protected float parseFloat(char[] buffer, int begin, int len) {
+        //TODO: pares float directly from char[]
+        String str = new String(buffer, begin, len);
+        return Float.valueOf(str);
+    }
+
+    protected int indexOf(char[] buffer, int begin, int len, char target) {
+        for (int i = begin; i < begin + len; i++) {
+            if (buffer[i] == target) {
+                return i;
+            }
+        }
+        throw new IllegalArgumentException("Cannot find " + target + " in " + new String(buffer, begin, len));
+    }
+
+    protected void parseString(char[] buffer, int begin, int length, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeByte(ATypeTag.STRING.serialize());
+            untaggedStringSerde.serialize(buffer, begin, length, out);
+        } catch (IOException e) {
+            throw new ParseException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
index ce8780d..209ba34 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
@@ -426,26 +426,27 @@
      * @throws IOException
      */
     private void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
-        final String stringValue = jsonParser.getText();
+        char[] buffer = jsonParser.getTextCharacters();
+        int begin = jsonParser.getTextOffset();
+        int len = jsonParser.getTextLength();
         final ATypeTag typeToUse = stringVariantType == ATypeTag.ANY ? currentToken().getTypeTag() : stringVariantType;
 
         switch (typeToUse) {
             case STRING:
-                aString.setValue(stringValue);
-                stringSerde.serialize(aString, out);
+                parseString(buffer, begin, len, out);
                 break;
             case DATE:
-                parseDate(stringValue, out);
+                parseDate(buffer, begin, len, out);
                 break;
             case DATETIME:
-                parseDateTime(stringValue, out);
+                parseDateTime(buffer, begin, len, out);
                 break;
             case TIME:
-                parseTime(stringValue, out);
+                parseTime(buffer, begin, len, out);
                 break;
             default:
                 throw new RuntimeDataException(ErrorCode.TYPE_UNSUPPORTED, jsonParser.currentToken().toString());
 
         }
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
new file mode 100644
index 0000000..333ac55
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.parser;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ParseException extends HyracksDataException {
+    private static final long serialVersionUID = 1L;
+    private String filename;
+    private int line = -1;
+    private int column = -1;
+
+    public ParseException(String message) {
+        super(message);
+    }
+
+    public ParseException(int errorCode, Serializable... param) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
+    }
+
+    public ParseException(int errorCode, Throwable e, Serializable... param) {
+        super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
+        addSuppressed(e);
+    }
+
+    public ParseException(Throwable cause) {
+        super(cause);
+    }
+
+    public ParseException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ParseException(Throwable cause, String filename, int line, int column) {
+        super(cause);
+        setLocation(filename, line, column);
+    }
+
+    public void setLocation(String filename, int line, int column) {
+        this.filename = filename;
+        this.line = line;
+        this.column = column;
+    }
+
+    @Override
+    public String getMessage() {
+        StringBuilder msg = new StringBuilder("Parse error");
+        if (filename != null) {
+            msg.append(" in file ").append(filename);
+        }
+        if (line >= 0) {
+            msg.append(" in line ").append(line);
+            if (column >= 0) {
+                msg.append(", at column ").append(column);
+            }
+        }
+        return msg.append(": ").append(super.getMessage()).toString();
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index bd50352..1b13ec5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -46,6 +46,7 @@
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -67,11 +68,11 @@
 
     public static IndexingScheduler initializeIndexingHDFSScheduler(ICCServiceContext serviceCtx)
             throws HyracksDataException {
-        ICCContext ccContext = serviceCtx.getCCContext();
         IndexingScheduler scheduler = null;
         try {
-            scheduler = new IndexingScheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
-                    ccContext.getClusterControllerInfo().getClientNetPort());
+            ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
+            IHyracksClientConnection hcc = appCtx.getHcc();
+            scheduler = new IndexingScheduler(hcc.getNodeControllerInfos());
         } catch (HyracksException e) {
             throw new RuntimeDataException(ErrorCode.UTIL_HDFS_UTILS_CANNOT_OBTAIN_HDFS_SCHEDULER);
         }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
index 1485f4f..189c842 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
@@ -18,60 +18,76 @@
  */
 package org.apache.asterix.external.parser.test;
 
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.external.parser.ADMDataParser;
 import org.apache.asterix.external.parser.AbstractDataParser;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMutableDate;
 import org.apache.asterix.om.base.AMutableDateTime;
 import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.esri.core.geometry.ogc.OGCPoint;
+
 public class ADMDataParserTest {
 
     @Test
     public void test() throws IOException, NoSuchMethodException, SecurityException, NoSuchFieldException {
-        String[] dates = { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" };
+        char[][] dates = toChars(
+                new String[] { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" });
         AMutableDate[] parsedDates =
                 new AMutableDate[] { new AMutableDate(-4202630), new AMutableDate(2807408), new AMutableDate(-4202752),
                         new AMutableDate(2807409), new AMutableDate(-4202569), new AMutableDate(2796544), };
 
-        String[] times = { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z", "04:16:42.321Z",
-                "12:22:56.816Z" };
+        char[][] times = toChars(new String[] { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z",
+                "04:16:42.321Z", "12:22:56.816Z" });
         AMutableTime[] parsedTimes =
                 new AMutableTime[] { new AMutableTime(43485689), new AMutableTime(45719002), new AMutableTime(43845169),
                         new AMutableTime(56268736), new AMutableTime(15402321), new AMutableTime(44576816), };
 
-        String[] dateTimes = { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
-                "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" };
+        char[][] dateTimes = toChars(
+                new String[] { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
+                        "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" });
         AMutableDateTime[] parsedDateTimes =
                 new AMutableDateTime[] { new AMutableDateTime(-145452954464325L), new AMutableDateTime(67345192751902L),
                         new AMutableDateTime(-45286270768513L), new AMutableDateTime(151729886421653L),
                         new AMutableDateTime(5047449515758L), new AMutableDateTime(210721439419691L) };
 
-        Method parseDateMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseDate", String.class, DataOutput.class);
+        Method parseDateMethod = AbstractDataParser.class.getDeclaredMethod("parseDate", char[].class, int.class,
+                int.class, DataOutput.class);
         parseDateMethod.setAccessible(true);
 
         Field aDateField = AbstractDataParser.class.getDeclaredField("aDate");
         aDateField.setAccessible(true);
 
-        Method parseTimeMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseTime", String.class, DataOutput.class);
+        Method parseTimeMethod = AbstractDataParser.class.getDeclaredMethod("parseTime", char[].class, int.class,
+                int.class, DataOutput.class);
         parseTimeMethod.setAccessible(true);
 
         Field aTimeField = AbstractDataParser.class.getDeclaredField("aTime");
         aTimeField.setAccessible(true);
 
-        Method parseDateTimeMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseDateTime", String.class, DataOutput.class);
+        Method parseDateTimeMethod = AbstractDataParser.class.getDeclaredMethod("parseDateTime", char[].class,
+                int.class, int.class, DataOutput.class);
         parseDateTimeMethod.setAccessible(true);
 
         Field aDateTimeField = AbstractDataParser.class.getDeclaredField("aDateTime");
@@ -92,21 +108,21 @@
                         while (round++ < 10000) {
                             // Test parseDate.
                             for (int index = 0; index < dates.length; ++index) {
-                                parseDateMethod.invoke(parser, dates[index], dos);
+                                parseDateMethod.invoke(parser, dates[index], 0, dates[index].length, dos);
                                 AMutableDate aDate = (AMutableDate) aDateField.get(parser);
                                 Assert.assertTrue(aDate.equals(parsedDates[index]));
                             }
 
                             // Tests parseTime.
                             for (int index = 0; index < times.length; ++index) {
-                                parseTimeMethod.invoke(parser, times[index], dos);
+                                parseTimeMethod.invoke(parser, times[index], 0, times[index].length, dos);
                                 AMutableTime aTime = (AMutableTime) aTimeField.get(parser);
                                 Assert.assertTrue(aTime.equals(parsedTimes[index]));
                             }
 
                             // Tests parseDateTime.
                             for (int index = 0; index < dateTimes.length; ++index) {
-                                parseDateTimeMethod.invoke(parser, dateTimes[index], dos);
+                                parseDateTimeMethod.invoke(parser, dateTimes[index], 0, dateTimes[index].length, dos);
                                 AMutableDateTime aDateTime = (AMutableDateTime) aDateTimeField.get(parser);
                                 Assert.assertTrue(aDateTime.equals(parsedDateTimes[index]));
                             }
@@ -132,4 +148,58 @@
         // Asserts no failure.
         Assert.assertTrue(errorCount.get() == 0);
     }
-}
\ No newline at end of file
+
+    private char[][] toChars(String[] strings) {
+        char[][] results = new char[strings.length][];
+        for (int i = 0; i < strings.length; i++) {
+            results[i] = strings[i].toCharArray();
+        }
+        return results;
+    }
+
+    @Test
+    public void testWKTParser() {
+        try {
+            ARecordType recordType = new ARecordType("POIType", new String[] { "id", "coord" },
+                    new IAType[] { BuiltinType.AINT32, BuiltinType.AGEOMETRY }, false);
+
+            String wktObject = "{\"id\": 123, \"coord\": \"POINT(3 4)\"}";
+            InputStream in = new ByteArrayInputStream(wktObject.getBytes());
+            ADMDataParser parser = new ADMDataParser(recordType, true);
+            parser.setInputStream(in);
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(baos);
+            parser.parse(out);
+            out.close();
+            byte[] serialized = baos.toByteArray();
+
+            // Parse to make sure it was correct
+            ByteBuffer bb = ByteBuffer.wrap(serialized);
+            Assert.assertEquals(ATypeTag.SERIALIZED_RECORD_TYPE_TAG, bb.get());
+            Assert.assertEquals(serialized.length, bb.getInt()); // Total record size including header
+            Assert.assertEquals(2, bb.getInt()); // # of records
+            int offsetOfID = bb.getInt();
+            int offsetOfGeometry = bb.getInt();
+            ISerializerDeserializer intDeser =
+                    SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT32);
+            Assert.assertEquals(offsetOfID, bb.position());
+            // Serialize the two records
+            DataInputByteBuffer dataIn = new DataInputByteBuffer();
+            dataIn.reset(bb);
+            Object o = intDeser.deserialize(dataIn);
+            Assert.assertEquals(new AInt32(123), o);
+            ISerializerDeserializer geomDeser =
+                    SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+            Object point = geomDeser.deserialize(dataIn);
+            Assert.assertTrue(point instanceof AGeometry);
+            Assert.assertTrue(((AGeometry) point).getGeometry() instanceof OGCPoint);
+            OGCPoint p = (OGCPoint) ((AGeometry) point).getGeometry();
+            Assert.assertEquals(3.0, p.X(), 1E-5);
+            Assert.assertEquals(4.0, p.Y(), 1E-5);
+        } catch (IOException e) {
+            e.printStackTrace();
+            Assert.fail("Error in parsing");
+        }
+
+    }
+}
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
index 4cc4975..62e4732 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,9 +54,9 @@
     protected final ISerializerDeserializer<ABoolean> booleanSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
-    public EditDistanceCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public EditDistanceCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         edThreshEval = args[2].createScalarEvaluator(context);
         listBuilder = new OrderedListBuilder();
         listItemVal = new ArrayBackedValueStorage();
@@ -110,7 +111,7 @@
             }
 
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE_CHECK, 0, argType.serialize(),
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE_CHECK, 0, argType.serialize(),
                         ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
 
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
index eaf3368..d35dd89 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 
 public class EditDistanceContainsEvaluator extends EditDistanceCheckEvaluator {
 
-    public EditDistanceContainsEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public EditDistanceContainsEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
@@ -51,8 +52,9 @@
                 return ed.getSimilarityContains(firstOrdListIter, secondOrdListIter, edThresh);
             }
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE_CONTAINS, 0, argType.serialize(),
-                        ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE_CONTAINS, 0,
+                        argType.serialize(), ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                        ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
         }
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
index c3af387..cc103e0 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,6 +54,7 @@
     protected final IPointable argPtr2 = new VoidPointable();
     protected final IScalarEvaluator firstStringEval;
     protected final IScalarEvaluator secondStringEval;
+    protected final SourceLocation sourceLoc;
     protected final SimilarityMetricEditDistance ed = new SimilarityMetricEditDistance();
     protected final OrderedListIterator firstOrdListIter = new OrderedListIterator();
     protected final OrderedListIterator secondOrdListIter = new OrderedListIterator();
@@ -66,10 +68,11 @@
     protected ATypeTag firstTypeTag;
     protected ATypeTag secondTypeTag;
 
-    public EditDistanceEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public EditDistanceEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
         firstStringEval = args[0].createScalarEvaluator(context);
         secondStringEval = args[1].createScalarEvaluator(context);
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -115,7 +118,7 @@
                 return (int) ed.computeSimilarity(firstOrdListIter, secondOrdListIter);
             }
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE, 0, argType.serialize(),
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, 0, argType.serialize(),
                         ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
 
@@ -124,13 +127,13 @@
 
     protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
         if (typeTag1 != typeTag2) {
-            throw new IncompatibleTypeException(BuiltinFunctions.EDIT_DISTANCE, typeTag1.serialize(),
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, typeTag1.serialize(),
                     typeTag2.serialize());
         }
 
         // Since they are equal, check one tag is enough.
         if (typeTag1 != ATypeTag.STRING && typeTag1 != ATypeTag.ARRAY) { // could be an list
-            throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE, 0, typeTag1.serialize(),
+            throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, 0, typeTag1.serialize(),
                     ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
         }
 
@@ -138,12 +141,14 @@
             itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
                     .deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset() + 1]);
             if (itemTypeTag == ATypeTag.ANY) {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE,
+                        itemTypeTag.serialize());
             }
             itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
                     .deserialize(argPtr2.getByteArray()[argPtr2.getStartOffset() + 1]);
             if (itemTypeTag == ATypeTag.ANY) {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE,
+                        itemTypeTag.serialize());
             }
         }
         return true;
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
index e4b40b1..d803a47 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceCheckEvaluator(args, ctx);
+                return new EditDistanceCheckEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
index 4c7c257..b6002b5 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceContainsEvaluator(args, ctx);
+                return new EditDistanceContainsEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
index 8c6c9ed..60c8867 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceEvaluator(args, ctx);
+                return new EditDistanceEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
index 17637bc..e7d582f 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
@@ -91,7 +91,7 @@
                         byte[] data = inputVal.getByteArray();
                         int offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length = IntegerPointable.getInteger(data, offset + 1);
@@ -101,7 +101,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -112,7 +112,7 @@
                         offset = inputVal.getStartOffset();
                         int len = inputVal.getLength();
                         if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, data[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         SimilarityFilters similarityFilters =
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
index 52bd8b1..d3fc6fc 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
@@ -94,7 +94,7 @@
                         byte[] data = thresholdPtr.getByteArray();
                         int offset = thresholdPtr.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
                         }
                         float similarityThreshold = AFloatSerializerDeserializer.getFloat(data, offset + 1);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index 8584d06..89da0a1 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -105,7 +105,7 @@
                         int offset = inputVal.getStartOffset();
 
                         if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -116,7 +116,7 @@
                         offset = inputVal.getStartOffset();
                         int len = inputVal.getLength();
                         if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         SimilarityFilters similarityFilters =
@@ -126,7 +126,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length1 = IntegerPointable.getInteger(data, offset + 1);
@@ -135,7 +135,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 3, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 3, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length2 = IntegerPointable.getInteger(data, offset + 1);
@@ -156,7 +156,7 @@
 
                             if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 4, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 4, data[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
@@ -192,7 +192,7 @@
 
                             if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 5, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 5, data[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index 4229f87..67e519f 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -122,7 +122,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         int counter = 0;
@@ -231,7 +232,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                         for (int i = 0; i < numOfPoints1; i++) {
                             double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
@@ -383,7 +385,7 @@
                         int v = middleVertex;
                         while (!foundEar) {
                             if (0 >= (nonSimplePolygonDetection--)) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
                             int u = v;
@@ -676,7 +678,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
@@ -728,7 +731,8 @@
                                 offset0 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints);
@@ -825,7 +829,7 @@
                                         res = pointInRectangle(bytes0, offset0, bytes1, offset1);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -894,7 +898,7 @@
                                         res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -916,7 +920,7 @@
                                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                                         if (numOfPoints0 < 3 || numOfPoints1 < 3) {
-                                            throw new InvalidDataFormatException(getIdentifier(),
+                                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                         }
 
@@ -987,7 +991,7 @@
                                         res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1012,7 +1016,7 @@
                                         res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1061,7 +1065,7 @@
                                         }
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1069,7 +1073,7 @@
                                 }
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
new file mode 100644
index 0000000..671e57e
--- /dev/null
+++ b/asterixdb/asterix-geo/pom.xml
@@ -0,0 +1,147 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>apache-asterixdb</artifactId>
+    <groupId>org.apache.asterix</groupId>
+    <version>0.9.4-SNAPSHOT</version>
+  </parent>
+  <artifactId>asterix-geo</artifactId>
+
+  <licenses>
+    <license>
+      <name>Apache License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+      <distribution>repo</distribution>
+      <comments>A business-friendly OSS license</comments>
+    </license>
+  </licenses>
+
+  <properties>
+    <root.dir>${basedir}/..</root.dir>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.asterix</groupId>
+        <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+        <version>${project.version}</version>
+        <configuration>
+          <evaluatorPackagePrefix>org.apache.asterix.runtime.evaluators</evaluatorPackagePrefix>
+        </configuration>
+        <executions>
+          <execution>
+            <id>generate-evaluator</id>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>generate-evaluator</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <excludes combine.children="append">
+            <exclude>data/**</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-data-std</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-runtime</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-om</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-runtime</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+      <version>2.0.0</version>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
new file mode 100644
index 0000000..99eaf46
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.aggregates;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * NULLs are also counted.
+ */
+public class STUnionAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STUnionAggregateDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_UNION_AGG;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STUnionAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
new file mode 100644
index 0000000..41c8c5b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.aggregates;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * STUnion aggregates a set of objects into one object. If the input is a set of overlapping polygons, their union is
+ * computed and returned as a multipolygon. Similarly, if the input is a set of points or linestring, a multipoint or
+ * multilinestring is created. Is the result contains geometries of different types, e.g., points and linestring, the
+ * output is a GeometryCollection.
+ */
+public class STUnionAggregateFunction extends AbstractAggregateFunction {
+    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AGeometry> geometrySerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY);
+    private IPointable inputVal = new VoidPointable();
+    private IScalarEvaluator eval;
+    protected OGCGeometry geometry;
+
+    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+    public STUnionAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
+        eval = args[0].createScalarEvaluator(context);
+    }
+
+    @Override
+    public void init() throws HyracksDataException {
+        // Initialize the resulting geometry with an empty point.
+        geometry = new OGCPoint(new Point(), DEFAULT_CRS);
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        eval.evaluate(tuple, inputVal);
+        byte[] data = inputVal.getByteArray();
+        int offset = inputVal.getStartOffset();
+        int len = inputVal.getLength();
+        ATypeTag typeTag =
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
+        // Ignore SYSTEM_NULL.
+        if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
+            processNull();
+        } else if (typeTag == ATypeTag.GEOMETRY) {
+            DataInput dataIn = new DataInputStream(new ByteArrayInputStream(data, offset + 1, len - 1));
+            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            geometry = geometry.union(geometry1);
+        }
+    }
+
+    @Override
+    public void finish(IPointable resultPointable) throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            geometrySerde.serialize(new AGeometry(geometry), resultStorage.getDataOutput());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+        resultPointable.set(resultStorage);
+    }
+
+    @Override
+    public void finishPartial(IPointable resultPointable) throws HyracksDataException {
+        finish(resultPointable);
+    }
+
+    protected void processNull() throws UnsupportedItemTypeException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.ST_UNION,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java
new file mode 100644
index 0000000..96b6478
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.aggregates;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.scalar.AbstractScalarAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSTUnionAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_ST_UNION_AGG;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ScalarSTUnionAggregateDescriptor(STUnionAggregateDescriptor.FACTORY.createFunctionDescriptor());
+        }
+    };
+
+    private ScalarSTUnionAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
new file mode 100644
index 0000000..1644b99
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators;
+
+import org.apache.asterix.geo.aggregates.STUnionAggregateDescriptor;
+import org.apache.asterix.geo.aggregates.ScalarSTUnionAggregateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.ParseGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAreaDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsBinaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STBoundaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STContainsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCoordDimDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCrossesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDimensionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDisjointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDistanceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEndPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEnvelopeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEqualsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STExteriorRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromWKBDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomentryTypeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeometryNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STInteriorRingNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsClosedDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsCollectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsEmptyDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsSimpleDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLengthDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLineFromMultiPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakeEnvelopeDescriptorSRID;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DWithMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNPointsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumGeometriesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumInteriorRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STOverlapsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPointNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPolygonizeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STRelateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STStartPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSymDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STTouchesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STUnionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STWithinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMinDescriptor;
+import org.apache.asterix.om.functions.IFunctionCollection;
+import org.apache.asterix.om.functions.IFunctionRegistrant;
+
+public class GeoFunctionRegistrant implements IFunctionRegistrant {
+    @Override
+    public void register(IFunctionCollection fc) {
+        //Geo functions
+        fc.add(ScalarSTUnionAggregateDescriptor.FACTORY);
+        fc.add(STUnionAggregateDescriptor.FACTORY);
+
+        //GeoJSON
+        fc.add(ParseGeoJSONDescriptor.FACTORY);
+        fc.add(STAreaDescriptor.FACTORY);
+        fc.add(STMakePointDescriptor.FACTORY);
+        fc.add(STMakePoint3DDescriptor.FACTORY);
+        fc.add(STMakePoint3DWithMDescriptor.FACTORY);
+        fc.add(STIntersectsDescriptor.FACTORY);
+        fc.add(STUnionDescriptor.FACTORY);
+        fc.add(STIsCollectionDescriptor.FACTORY);
+        fc.add(STContainsDescriptor.FACTORY);
+        fc.add(STCrossesDescriptor.FACTORY);
+        fc.add(STDisjointDescriptor.FACTORY);
+        fc.add(STEqualsDescriptor.FACTORY);
+        fc.add(STOverlapsDescriptor.FACTORY);
+        fc.add(STTouchesDescriptor.FACTORY);
+        fc.add(STWithinDescriptor.FACTORY);
+        fc.add(STIsEmptyDescriptor.FACTORY);
+        fc.add(STIsSimpleDescriptor.FACTORY);
+        fc.add(STCoordDimDescriptor.FACTORY);
+        fc.add(STDimensionDescriptor.FACTORY);
+        fc.add(STGeomentryTypeDescriptor.FACTORY);
+        fc.add(STMDescriptor.FACTORY);
+        fc.add(STNRingsDescriptor.FACTORY);
+        fc.add(STNPointsDescriptor.FACTORY);
+        fc.add(STNumGeometriesDescriptor.FACTORY);
+        fc.add(STNumInteriorRingsDescriptor.FACTORY);
+        fc.add(STSRIDDescriptor.FACTORY);
+        fc.add(STXDescriptor.FACTORY);
+        fc.add(STYDescriptor.FACTORY);
+        fc.add(STXMaxDescriptor.FACTORY);
+        fc.add(STXMinDescriptor.FACTORY);
+        fc.add(STYMaxDescriptor.FACTORY);
+        fc.add(STYMinDescriptor.FACTORY);
+        fc.add(STZDescriptor.FACTORY);
+        fc.add(STZMaxDescriptor.FACTORY);
+        fc.add(STZMinDescriptor.FACTORY);
+        fc.add(STAsBinaryDescriptor.FACTORY);
+        fc.add(STAsTextDescriptor.FACTORY);
+        fc.add(STAsGeoJSONDescriptor.FACTORY);
+        fc.add(STDistanceDescriptor.FACTORY);
+        fc.add(STLengthDescriptor.FACTORY);
+        fc.add(STGeomFromTextDescriptor.FACTORY);
+        fc.add(STGeomFromTextSRIDDescriptor.FACTORY);
+        fc.add(STGeomFromWKBDescriptor.FACTORY);
+        fc.add(STLineFromMultiPointDescriptor.FACTORY);
+        fc.add(STMakeEnvelopeDescriptorSRID.FACTORY);
+        fc.add(STIsClosedDescriptor.FACTORY);
+        fc.add(STIsRingDescriptor.FACTORY);
+        fc.add(STRelateDescriptor.FACTORY);
+        fc.add(STBoundaryDescriptor.FACTORY);
+        fc.add(STEndPointDescriptor.FACTORY);
+        fc.add(STEnvelopeDescriptor.FACTORY);
+        fc.add(STExteriorRingDescriptor.FACTORY);
+        fc.add(STGeometryNDescriptor.FACTORY);
+        fc.add(STInteriorRingNDescriptor.FACTORY);
+        fc.add(STPointNDescriptor.FACTORY);
+        fc.add(STStartPointDescriptor.FACTORY);
+        fc.add(STDifferenceDescriptor.FACTORY);
+        fc.add(STIntersectionDescriptor.FACTORY);
+        fc.add(STSymDifferenceDescriptor.FACTORY);
+        fc.add(STPolygonizeDescriptor.FACTORY);
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java
new file mode 100644
index 0000000..f609beb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators;
+
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class GeoFunctionTypeInferers {
+    private GeoFunctionTypeInferers() {
+    }
+
+    public static final class GeometryConstructorTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("parse-geojson for data of type " + t);
+            }
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
new file mode 100644
index 0000000..65992f7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+
+public abstract class AbstractGetValDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    public double getVal(byte[] bytes, int offset) throws TypeMismatchException {
+        if (bytes[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+            return ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
+        } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+            return AInt64SerializerDeserializer.getLong(bytes, offset + 1);
+        } else {
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
+                    ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
new file mode 100644
index 0000000..60f6ea4
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTDoubleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1)
+            throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractSTDoubleGeometryEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTDoubleGeometryEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private final IPointable argPtr0;
+        private final IPointable argPtr1;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+
+        public AbstractSTDoubleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            argPtr0 = new VoidPointable();
+            argPtr1 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            resultStorage.reset();
+            eval0.evaluate(tuple, argPtr0);
+            eval1.evaluate(tuple, argPtr1);
+
+            try {
+                byte[] bytes0 = argPtr0.getByteArray();
+                int offset0 = argPtr0.getStartOffset();
+                int len0 = argPtr0.getLength();
+                byte[] bytes1 = argPtr1.getByteArray();
+                int offset1 = argPtr1.getStartOffset();
+                int len1 = argPtr1.getLength();
+
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+                tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+
+                DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+                DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+                OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+                Object finalResult = evaluateOGCGeometry(geometry0, geometry1);
+                if (finalResult instanceof OGCGeometry) {
+                    out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                } else if (finalResult instanceof Boolean) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                            .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+                } else if (finalResult instanceof Double) {
+                    out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                    out.writeDouble((double) finalResult);
+                }
+
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
new file mode 100644
index 0000000..cf1a900
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTGeometryNDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    abstract protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new AbstractSTGeometryNEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTGeometryNEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+
+        public AbstractSTGeometryNEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            if (data[offset] != ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            int n = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+
+            OGCGeometry geometryN = evaluateOGCGeometry(geometry, n);
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry(geometryN), out);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
new file mode 100644
index 0000000..0100261
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTSingleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            @SuppressWarnings("unchecked")
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractSTSingleGeometryEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTSingleGeometryEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private final IPointable argPtr0;
+        private final IScalarEvaluator eval0;
+
+        private final AMutableInt32 intRes;
+
+        public AbstractSTSingleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            argPtr0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            intRes = new AMutableInt32(0);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            resultStorage.reset();
+            eval0.evaluate(tuple, argPtr0);
+
+            try {
+                byte[] bytes0 = argPtr0.getByteArray();
+                int offset0 = argPtr0.getStartOffset();
+                int len0 = argPtr0.getLength();
+
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+
+                DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+
+                Object finalResult = evaluateOGCGeometry(geometry0);
+                if (finalResult == null) {
+                    out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                } else if (finalResult instanceof Double) {
+                    out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                    out.writeDouble((double) finalResult);
+                } else if (finalResult instanceof Boolean) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                            .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+                } else if (finalResult instanceof Integer) {
+                    intRes.setValue((int) finalResult);
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32)
+                            .serialize(intRes, out);
+                } else if (finalResult instanceof String) {
+                    out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                    out.write(UTF8StringUtil.writeStringToBytes((String) finalResult));
+                } else if (finalResult instanceof byte[]) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABINARY)
+                            .serialize(new ABinary((byte[]) finalResult), out);
+                } else if (finalResult instanceof OGCGeometry) {
+                    out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
new file mode 100644
index 0000000..75b3331
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.MapOGCStructure;
+import com.esri.core.geometry.OperatorImportFromGeoJson;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.geo.evaluators.GeoFunctionTypeInferers;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class ParseGeoJSONDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ParseGeoJSONDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new GeoFunctionTypeInferers.GeometryConstructorTypeInferer();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        this.recType = (ARecordType) states[0];
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GEOMETRY_CONSTRUCTOR;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new ParseGeoJSONEvaluator(args[0], ctx);
+            }
+        };
+    }
+
+    private class ParseGeoJSONEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private OperatorImportFromGeoJson geoJsonImporter;
+
+        public ParseGeoJSONEvaluator(IScalarEvaluatorFactory factory, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = factory.createScalarEvaluator(ctx);
+            geoJsonImporter = OperatorImportFromGeoJson.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GEOMETRY_CONSTRUCTOR, 0, data[offset],
+                        ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+            }
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInput dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = recordToString(new ARecordSerializerDeserializer(recType).deserialize(dataIn));
+                MapOGCStructure structure = geoJsonImporter.executeOGC(0, geometry, null);
+                OGCGeometry ogcGeometry =
+                        OGCGeometry.createFromOGCStructure(structure.m_ogcStructure, structure.m_spatialReference);
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+
+        public String recordToString(ARecord record) {
+            StringBuilder sb = new StringBuilder();
+            sb.append("{ ");
+            String[] fieldNames = record.getType().getFieldNames();
+            IAObject val;
+            if (fieldNames != null) {
+                for (int i = 0; i < fieldNames.length; i++) {
+                    if (i > 0) {
+                        sb.append(", ");
+                    }
+                    sb.append("\"").append(fieldNames[i]).append("\"").append(": ");
+                    val = record.getValueByPos(i);
+                    if (val instanceof ARecord) {
+                        sb.append(recordToString((ARecord) val));
+                    } else if (val instanceof AOrderedList) {
+                        sb.append(listToString((AOrderedList) val));
+                    } else {
+                        sb.append(val);
+                    }
+                }
+            }
+            sb.append(" }");
+            return sb.toString();
+        }
+
+        public String listToString(AOrderedList list) {
+            StringBuilder sb = new StringBuilder();
+            boolean first = true;
+            IAObject val;
+            sb.append("[ ");
+            for (int i = 0; i < list.size(); i++) {
+                if (first) {
+                    first = false;
+                } else {
+                    sb.append(", ");
+                }
+                val = list.getItem(i);
+                if (val instanceof ARecord) {
+                    sb.append(recordToString((ARecord) val));
+                } else if (val instanceof AOrderedList) {
+                    sb.append(listToString((AOrderedList) val));
+                } else {
+                    sb.append(val);
+                }
+            }
+            sb.append(" ]");
+            return sb.toString();
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
new file mode 100644
index 0000000..caf0b89
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAreaDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAreaDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        double area;
+        if (!"GeometryCollection".equals(geometry.geometryType())) {
+            area = geometry.getEsriGeometry().calculateArea2D();
+        } else {
+            GeometryCursor cursor = geometry.getEsriGeometryCursor();
+            Geometry geometry1 = cursor.next();
+            area = 0;
+            while (geometry1 != null) {
+                area += geometry1.calculateArea2D();
+                geometry1 = cursor.next();
+            }
+        }
+        return area;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AREA;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
new file mode 100644
index 0000000..387d07c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsBinaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsBinaryDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asBinary().array();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_BINARY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
new file mode 100644
index 0000000..b12b851
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsGeoJSONDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsGeoJSONDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asGeoJson();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_GEOJSON;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
new file mode 100644
index 0000000..3e310d9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsTextDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsTextDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asText();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_TEXT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
new file mode 100644
index 0000000..55c5212
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STBoundaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STBoundaryDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.boundary();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_BOUNDARY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
new file mode 100644
index 0000000..f9f692f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STContainsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STContainsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.contains(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_CONTAINS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
new file mode 100644
index 0000000..d1a3efd
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCoordDimDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STCoordDimDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.coordinateDimension();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_COORD_DIM;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
new file mode 100644
index 0000000..1ac67fe
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCrossesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STCrossesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.crosses(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_CROSSES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
new file mode 100644
index 0000000..a0209f9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDifferenceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.difference(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DIFFERENCE;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
new file mode 100644
index 0000000..492d045
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDimensionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDimensionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.dimension();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DIMENSION;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
new file mode 100644
index 0000000..17272ce
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDisjointDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDisjointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.disjoint(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DISJOINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
new file mode 100644
index 0000000..50cb129
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDistanceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDistanceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.distance(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DISTANCE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
new file mode 100644
index 0000000..6b8e43b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEndPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEndPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).endPoint();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_END_POINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
new file mode 100644
index 0000000..f5d7d0d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEnvelopeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEnvelopeDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.envelope();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_ENVELOPE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
new file mode 100644
index 0000000..66c2ab1
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEqualsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEqualsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.equals(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_EQUALS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
new file mode 100644
index 0000000..bda2c90
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STExteriorRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STExteriorRingDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).exteriorRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_EXTERIOR_RING;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
new file mode 100644
index 0000000..81fa68d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromTextDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_TEXT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromTextEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromTextEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private OperatorImportFromWkt wktImporter;
+
+        public STGeomFromTextEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            wktImporter = OperatorImportFromWkt.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_GEOM_FROM_TEXT, 0, data[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                OGCStructure structure;
+
+                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
new file mode 100644
index 0000000..42401c5
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextSRIDDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromTextSRIDDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_TEXT_SRID;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromTextSRIDEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromTextSRIDEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private OperatorImportFromWkt wktImporter;
+
+        public STGeomFromTextSRIDEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[1].createScalarEvaluator(ctx);
+            wktImporter = OperatorImportFromWkt.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                int srid = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+                OGCStructure structure;
+
+                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(srid));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
new file mode 100644
index 0000000..e949b90
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STGeomFromWKBDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromWKBDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_WKB;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromWKBEvaulator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromWKBEvaulator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+
+        public STGeomFromWKBEvaulator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_BINARY_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_GEOM_FROM_WKB, 0, data[offset],
+                        ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
+            }
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+                DataInputStream dataIn = new DataInputStream(inStream);
+                data = ByteArraySerializerDeserializer.read(dataIn);
+                out.writeInt(data.length);
+                out.write(data);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
new file mode 100644
index 0000000..8e7ae39
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeomentryTypeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomentryTypeDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.geometryType();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GEOMETRY_TYPE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
new file mode 100644
index 0000000..9bf3699
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeometryNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeometryNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOMETRY_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCGeometryCollection) {
+            return ((OGCGeometryCollection) geometry).geometryN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
new file mode 100644
index 0000000..25df415
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STInteriorRingNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STInteriorRingNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERIOR_RING_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).interiorRingN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
new file mode 100644
index 0000000..6f3314e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIntersectionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.intersection(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERSECTION;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
new file mode 100644
index 0000000..1a4f37f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIntersectsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.intersects(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERSECTS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
new file mode 100644
index 0000000..59edb4e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import com.esri.core.geometry.ogc.OGCMultiCurve;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPoint;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsClosedDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsClosedDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return isClosed(geometry);
+    }
+
+    private boolean isClosed(OGCGeometry geometry) {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).isClosed();
+        } else if (geometry instanceof OGCMultiCurve) {
+            return ((OGCMultiCurve) geometry).isClosed();
+        } else if (geometry instanceof OGCMultiPoint || geometry instanceof OGCMultiPolygon
+                || geometry instanceof OGCPolygon || geometry instanceof OGCPoint) {
+            return true;
+        } else if (geometry instanceof OGCGeometryCollection) {
+            OGCGeometryCollection geometryCollection = (OGCGeometryCollection) geometry;
+            int num = geometryCollection.numGeometries();
+            for (int i = 0; i < num; ++i) {
+                if (!isClosed(geometryCollection.geometryN(i))) {
+                    return false;
+                }
+            }
+            return true;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_CLOSED;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
new file mode 100644
index 0000000..92767e9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsCollectionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsCollectionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        String type = geometry.geometryType();
+        return "GeometryCollection".equals(type) || "MultiLineString".equals(type) || "MultiPoint".equals(type)
+                || "MultiPolygon".equals(type);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_COLLECTION;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
new file mode 100644
index 0000000..acf20b7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsEmptyDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsEmptyDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.isEmpty();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_EMPTY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
new file mode 100644
index 0000000..3c9375a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsRingDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).isRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_RING;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
new file mode 100644
index 0000000..fdb948f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsSimpleDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsSimpleDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.isSimple();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_SIMPLE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
new file mode 100644
index 0000000..65fd085
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLengthDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STLengthDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCLineString) {
+            return geometry.getEsriGeometry().calculateLength2D();
+        } else if (geometry instanceof OGCMultiLineString) {
+            GeometryCursor cursor = geometry.getEsriGeometryCursor();
+            double length = 0;
+            Geometry geometry1 = cursor.next();
+            while (geometry1 != null) {
+                length += geometry1.calculateLength2D();
+                geometry1 = cursor.next();
+            }
+            return length;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_LENGTH;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
new file mode 100644
index 0000000..c783fe0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.Polyline;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLineFromMultiPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STLineFromMultiPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCMultiPoint) {
+            Polyline polyline = new Polyline();
+            OGCMultiPoint multiPoint = (OGCMultiPoint) geometry;
+            int numPoints = multiPoint.numGeometries();
+            polyline.startPath((Point) multiPoint.geometryN(0).getEsriGeometry());
+            for (int i = 1; i < numPoints; i++) {
+                polyline.lineTo((Point) multiPoint.geometryN(i).getEsriGeometry());
+            }
+            return new OGCLineString(polyline, 0, SpatialReference.create(4326));
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_LINE_FROM_MULTIPOINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
new file mode 100644
index 0000000..46ce0fb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STMDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).M();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_M;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
new file mode 100644
index 0000000..5fa6da8
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STMakeEnvelopeDescriptorSRID extends AbstractGetValDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakeEnvelopeDescriptorSRID();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_ENVELOPE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STMakeEnvelopeEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STMakeEnvelopeEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private IPointable inputArg1;
+        private IScalarEvaluator eval1;
+        private IPointable inputArg2;
+        private IScalarEvaluator eval2;
+        private IPointable inputArg3;
+        private IScalarEvaluator eval3;
+        private IPointable inputArg4;
+        private IScalarEvaluator eval4;
+
+        public STMakeEnvelopeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            inputArg1 = new VoidPointable();
+            eval1 = args[1].createScalarEvaluator(ctx);
+            inputArg2 = new VoidPointable();
+            eval2 = args[2].createScalarEvaluator(ctx);
+            inputArg3 = new VoidPointable();
+            eval3 = args[3].createScalarEvaluator(ctx);
+            inputArg4 = new VoidPointable();
+            eval4 = args[4].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            eval1.evaluate(tuple, inputArg1);
+            byte[] data1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+
+            eval2.evaluate(tuple, inputArg2);
+            byte[] data2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+
+            eval3.evaluate(tuple, inputArg3);
+            byte[] data3 = inputArg3.getByteArray();
+            int offset3 = inputArg3.getStartOffset();
+
+            eval4.evaluate(tuple, inputArg4);
+            byte[] data4 = inputArg4.getByteArray();
+            int offset4 = inputArg4.getStartOffset();
+
+            try {
+
+                OGCGeometry ogcGeometry =
+                        OGCGeometry
+                                .createFromEsriGeometry(
+                                        new Envelope(getVal(data0, offset0), getVal(data1, offset1),
+                                                getVal(data2, offset2), getVal(data3, offset3)),
+                                        SpatialReference.create((int) getVal(data4, offset4)));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] bytes = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(bytes.length);
+                out.write(bytes);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
new file mode 100644
index 0000000..3c0ac97
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePoint3DDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePoint3DEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT3D;
+    }
+
+    private class STMakePoint3DEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private IPointable inputArg2;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private final IScalarEvaluator eval2;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePoint3DEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            inputArg2 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            eval2 = args[2].createScalarEvaluator(ctx);
+            point = new Point(0, 0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+            eval2.evaluate(tuple, inputArg2);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            byte[] bytes2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                point.setZ(getVal(bytes2, offset2));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
new file mode 100644
index 0000000..424fb8f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DWithMDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePoint3DWithMDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePoint3DWithMEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT3D_M;
+    }
+
+    private class STMakePoint3DWithMEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private IPointable inputArg2;
+        private IPointable inputArg3;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private final IScalarEvaluator eval2;
+        private final IScalarEvaluator eval3;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePoint3DWithMEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            inputArg2 = new VoidPointable();
+            inputArg3 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            eval2 = args[2].createScalarEvaluator(ctx);
+            eval3 = args[3].createScalarEvaluator(ctx);
+            point = new Point(0, 0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+            eval2.evaluate(tuple, inputArg2);
+            eval3.evaluate(tuple, inputArg3);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            byte[] bytes2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+            byte[] bytes3 = inputArg3.getByteArray();
+            int offset3 = inputArg3.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                point.setZ(getVal(bytes2, offset2));
+                point.setM(getVal(bytes3, offset3));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
new file mode 100644
index 0000000..2d901a7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePointDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePointDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePointEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT;
+    }
+
+    private class STMakePointEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePointEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            point = new Point(0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
new file mode 100644
index 0000000..8b1111b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.MultiVertexGeometry;
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNPointsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNPointsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeometry = geometry.getEsriGeometry();
+        if (esriGeometry != null && esriGeometry instanceof MultiVertexGeometry) {
+            return ((MultiVertexGeometry) esriGeometry).getPointCount();
+        } else if (esriGeometry instanceof Point) {
+            return 1;
+        } else if (esriGeometry == null) {
+            int count = 0;
+            GeometryCursor geometryCursor = geometry.getEsriGeometryCursor();
+            esriGeometry = geometryCursor.next();
+            while (esriGeometry != null) {
+                if (esriGeometry instanceof MultiVertexGeometry) {
+                    count += ((MultiVertexGeometry) esriGeometry).getPointCount();
+                } else if (esriGeometry instanceof Point) {
+                    count += 1;
+                }
+                esriGeometry = geometryCursor.next();
+            }
+            return count;
+        } else if (geometry.isEmpty()) {
+            return 0;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_N_POINTS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
new file mode 100644
index 0000000..27e0490
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNRingsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).numInteriorRing() + 1;
+        } else if (geometry instanceof OGCMultiPolygon) {
+            OGCMultiPolygon polygon = (OGCMultiPolygon) geometry;
+            int numGeometries = polygon.numGeometries();
+            int count = 0;
+            for (int i = 1; i < numGeometries + 1; i++) {
+                if (polygon.geometryN(i) instanceof OGCPolygon) {
+                    count += ((OGCPolygon) polygon.geometryN(i)).numInteriorRing() + 1;
+                }
+            }
+            return count;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_N_RINGS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
new file mode 100644
index 0000000..b8d0c52
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumGeometriesDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNumGeometriesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCGeometryCollection) {
+            return ((OGCGeometryCollection) geometry).numGeometries();
+        } else if (!geometry.isEmpty()) {
+            return 1;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_NUM_GEOMETRIIES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
new file mode 100644
index 0000000..0c32ec5
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumInteriorRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNumInteriorRingsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).numInteriorRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_NUM_INTERIOR_RINGS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
new file mode 100644
index 0000000..f131499
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STOverlapsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STOverlapsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.overlaps(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_OVERLAPS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
new file mode 100644
index 0000000..e3ce3e0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STPointNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STPointNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_POINT_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCLineString) {
+            return ((OGCLineString) geometry).pointN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
new file mode 100644
index 0000000..12024bc
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCConcreteGeometryCollection;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class STPolygonizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STPolygonizeDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_POLYGONIZE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STPolygonizeEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STPolygonizeEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+
+        public STPolygonizeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] bytes = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            AOrderedListType type = new AOrderedListType(BuiltinType.AGEOMETRY, null);
+            byte typeTag = inputArg.getByteArray()[inputArg.getStartOffset()];
+            ISerializerDeserializer serde;
+            if (typeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+                serde = new AOrderedListSerializerDeserializer(type);
+            } else if (typeTag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+                serde = new AOrderedListSerializerDeserializer(type);
+            } else {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_POLYGONIZE, 0, typeTag,
+                        ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            IACursor cursor = ((IACollection) serde.deserialize(dataIn)).getCursor();
+            List<OGCGeometry> list = new ArrayList<>();
+            while (cursor.next()) {
+                IAObject object = cursor.get();
+                list.add(((AGeometry) object).getGeometry());
+            }
+            OGCGeometryCollection geometryCollection =
+                    new OGCConcreteGeometryCollection(list, SpatialReference.create(4326));
+            try {
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY)
+                        .serialize(new AGeometry(geometryCollection), out);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
new file mode 100644
index 0000000..5745dca
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STRelateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STRelateDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_RELATE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STRelateEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STRelateEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private final IPointable inputArg1;
+        private final IScalarEvaluator eval1;
+
+        public STRelateEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[2].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            inputArg1 = new VoidPointable();
+            eval1 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] bytes = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            int len0 = inputArg0.getLength();
+
+            eval1.evaluate(tuple, inputArg1);
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            int len1 = inputArg1.getLength();
+
+            if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+            if (tag != ATypeTag.GEOMETRY) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+            if (tag != ATypeTag.GEOMETRY) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            String matrix = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+            DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+            OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+            DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+            try {
+                boolean val = geometry0.relate(geometry1, matrix);
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                        .serialize(val ? ABoolean.TRUE : ABoolean.FALSE, out);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
new file mode 100644
index 0000000..a46749d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSRIDDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STSRIDDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.SRID();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_SRID;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
new file mode 100644
index 0000000..988e781
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STStartPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STStartPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).startPoint();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_START_POINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
new file mode 100644
index 0000000..e92f56b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSymDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STSymDifferenceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.symDifference(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_SYM_DIFFERENCE;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
new file mode 100644
index 0000000..93a947a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STTouchesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STTouchesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.touches(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_TOUCHES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
new file mode 100644
index 0000000..43891f0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STUnionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STUnionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.union(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_UNION;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
new file mode 100644
index 0000000..f563d61
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STWithinDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STWithinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.within(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_WITHIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
new file mode 100644
index 0000000..c796b1b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).X();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
new file mode 100644
index 0000000..19864bb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getXMax();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
new file mode 100644
index 0000000..12378c6
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getXMin();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
new file mode 100644
index 0000000..782370d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).Y();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
new file mode 100644
index 0000000..83ea647
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getYMax();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
new file mode 100644
index 0000000..c6b5f36
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getYMin();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
new file mode 100644
index 0000000..b5b6d78
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).Z();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
new file mode 100644
index 0000000..7933f08
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.MaxZ();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
new file mode 100644
index 0000000..d0da92e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.MinZ();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
new file mode 100644
index 0000000..d9529e2
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+org.apache.asterix.geo.evaluators.GeoFunctionRegistrant
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
index bb4d346..1174717 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
@@ -22,11 +22,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DistinctClause implements Clause {
+public class DistinctClause extends AbstractClause {
 
     private List<Expression> distinctByExprs;
 
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
index 6cdc2f3..9bae2a0 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
@@ -20,12 +20,12 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class ForClause implements Clause {
+public class ForClause extends AbstractClause {
     private VariableExpr varExpr = null;
     private VariableExpr posExpr = null;
     private Expression inExpr = null;
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
index 4767f4f..f0ed087 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
@@ -22,11 +22,12 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLPlusVisitor;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class JoinClause implements Clause {
+public class JoinClause extends AbstractClause {
 
     public static enum JoinKind {
         INNER,
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
index 871c2ce..7712cf9 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLPlusVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class MetaVariableClause implements Clause {
+public class MetaVariableClause extends AbstractClause {
     private VarIdentifier var;
 
     @Override
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
index d2cb64f..dd7061f 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
@@ -77,7 +77,7 @@
         }
         inlineDeclaredUdfs();
         rewriteFunctionName();
-        topStatement.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter().get());
     }
 
     private void wrapInLets() {
@@ -119,7 +119,7 @@
         for (Expression topLevelExpr : topStatement.getDirectlyEnclosedExpressions()) {
             storedFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr, funIds,
                     null, expr -> getFunctionCalls(expr), func -> functionParser.getFunctionDecl(func),
-                    signature -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature)));
+                    (signature, sourceLoc) -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature)));
             declaredFunctions.addAll(storedFunctionDecls);
         }
         if (!declaredFunctions.isEmpty()) {
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 2d87556..856073d 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -147,6 +147,7 @@
 import org.apache.asterix.lang.common.statement.UpsertStatement;
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -1811,23 +1812,25 @@
 Expression MultExpr()throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = ExponentExpr()
 
-    (( <MUL> | <DIV> | <MOD> | <IDIV>)
+    ( (
+        <MUL> { opType = OperatorType.MUL; } |
+        <DIVIDE> { opType = OperatorType.DIVIDE; } |
+        <DIV> { opType = OperatorType.DIV; } |
+        ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+      )
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
         }
-        try{
-          op.addOperator(token.image);
-        } catch (CompilationException e){
-          throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
     operand = ExponentExpr()
     {
@@ -2724,10 +2727,10 @@
 TOKEN :
 {
     <CARET : "^">
-  | <DIV : "/">
-  | <IDIV : "idiv">
+  | <DIVIDE : "/">
+  | <DIV : "div">
   | <MINUS : "-">
-  | <MOD : "%">
+  | <MOD : "mod">
   | <MUL : "*">
   | <PLUS : "+">
 
@@ -2739,6 +2742,7 @@
   | <COLON : ":">
   | <COMMA : ",">
   | <DOT : ".">
+  | <PERCENT: "%">
   | <QUES : "?">
 
   | <LT : "<">
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java
new file mode 100644
index 0000000..bc7b45b
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.base;
+
+public abstract class AbstractClause extends AbstractLangExpression implements Clause {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
index 44a05ef..773268f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
@@ -23,7 +23,8 @@
 
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 
-public abstract class AbstractExpression implements Expression {
+public abstract class AbstractExpression extends AbstractLangExpression implements Expression {
+
     protected List<IExpressionAnnotation> hints;
 
     public void addHint(IExpressionAnnotation hint) {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java
new file mode 100644
index 0000000..3a784e3
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.base;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractLangExpression implements ILangExpression {
+    private SourceLocation sourceLoc;
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java
new file mode 100644
index 0000000..e07683d
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.base;
+
+public abstract class AbstractStatement extends AbstractLangExpression implements Statement {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
index f6b9399..e80d7bb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
@@ -20,7 +20,10 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILangExpression {
-    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException;
+    <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException;
+
+    SourceLocation getSourceLocation();
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
index 31c903e..20dba52 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
@@ -24,7 +24,7 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -32,7 +32,7 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class GroupbyClause implements Clause {
+public class GroupbyClause extends AbstractClause {
 
     private List<GbyVariableExpressionPair> gbyPairList;
     private List<GbyVariableExpressionPair> decorPairList;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
index df560ea..750cbe3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
@@ -21,12 +21,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LetClause implements Clause {
+public class LetClause extends AbstractClause {
     private VariableExpr varExpr;
     private Expression bindExpr;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
index fbfa64e..d597bc2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
@@ -21,11 +21,11 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LimitClause implements Clause {
+public class LimitClause extends AbstractClause {
     private Expression limitExpr;
     private Expression offset;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index c1ceaa8..faefe1d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -22,12 +22,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 
-public class OrderbyClause implements Clause {
+public class OrderbyClause extends AbstractClause {
     private List<Expression> orderbyList;
     private List<OrderModifier> modifierList;
     private IRangeMap rangeMap;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
index 4089c07..49c0c58 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
@@ -21,14 +21,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.statement.DeleteStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UpdateClause implements Clause {
+public class UpdateClause extends AbstractClause {
 
     private Expression target;
     private Expression value;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
index c540588..9153400 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
@@ -19,11 +19,11 @@
 package org.apache.asterix.lang.common.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class WhereClause implements Clause {
+public class WhereClause extends AbstractClause {
     private Expression whereExpr;
 
     public WhereClause() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java
new file mode 100644
index 0000000..f55f055
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.expression;
+
+import org.apache.asterix.lang.common.base.AbstractLangExpression;
+
+public abstract class AbstractTypeExpression extends AbstractLangExpression implements TypeExpression {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
index 0c3fdb1..ab6b3f1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
@@ -21,19 +21,9 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.AbstractExpression;
 import org.apache.asterix.lang.common.base.Literal;
-import org.apache.asterix.lang.common.literal.DoubleLiteral;
-import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
-import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.object.base.AdmBigIntNode;
-import org.apache.asterix.object.base.AdmBooleanNode;
-import org.apache.asterix.object.base.AdmDoubleNode;
-import org.apache.asterix.object.base.AdmNullNode;
-import org.apache.asterix.object.base.AdmStringNode;
-import org.apache.asterix.object.base.IAdmNode;
 
 public class LiteralExpr extends AbstractExpression {
     private Literal value;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
index 90b457e..9521469 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
@@ -84,10 +84,18 @@
         exprList.add(operand);
     }
 
-    public static final boolean opIsComparison(OperatorType t) {
-        boolean cmp = t == OperatorType.EQ || t == OperatorType.NEQ || t == OperatorType.GT;
-        cmp = cmp || t == OperatorType.GE || t == OperatorType.LT || t == OperatorType.LE;
-        return cmp;
+    public static boolean opIsComparison(OperatorType t) {
+        switch (t) {
+            case EQ:
+            case NEQ:
+            case GT:
+            case GE:
+            case LT:
+            case LE:
+                return true;
+            default:
+                return false;
+        }
     }
 
     public void addOperator(String strOp) throws CompilationException {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
index 58d3756..d4c03b2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class OrderedListTypeDefinition implements TypeExpression {
+public class OrderedListTypeDefinition extends AbstractTypeExpression {
 
     private TypeExpression itemTypeExpression;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
index 4aecd92..4fcc7ae 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class RecordTypeDefinition implements TypeExpression {
+public class RecordTypeDefinition extends AbstractTypeExpression {
 
     public enum RecordKind {
         OPEN,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
index 8c1e5c6..a76e5bb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class TypeReferenceExpression implements TypeExpression {
+public class TypeReferenceExpression extends AbstractTypeExpression {
 
     private final Pair<Identifier, Identifier> ident;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
index 8f4b1b7..b55c910 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UnorderedListTypeDefinition implements TypeExpression {
+public class UnorderedListTypeDefinition extends AbstractTypeExpression {
 
     private TypeExpression itemTypeExpression;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 10cd7d8..4173563 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -21,25 +21,21 @@
 import java.util.HashMap;
 
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 
 public final class LangRewritingContext {
-    private int mark = 0;
-    private int varCounter;
+    private Counter varCounter;
     private int systemVarCounter = 1;
     private HashMap<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
 
     public LangRewritingContext(int varCounter) {
-        this.varCounter = varCounter;
+        this.varCounter = new Counter(varCounter);
     }
 
-    public int getVarCounter() {
+    public Counter getVarCounter() {
         return varCounter;
     }
 
-    public void setVarCounter(int varCounter) {
-        this.varCounter = varCounter;
-    }
-
     /**
      * Generate a new variable with the same identifier (varValue) but a different Id.
      *
@@ -50,7 +46,7 @@
      * @return the new varible.
      */
     public VarIdentifier mapOldId(Integer oldId, String varValue) {
-        int n = newId();
+        int n = increaseAndGetCtr();
         VarIdentifier newVar = new VarIdentifier(varValue);
         newVar.setId(n);
         oldVarIdToNewVarId.put(oldId, newVar);
@@ -62,22 +58,14 @@
     }
 
     public VarIdentifier newVariable() {
-        int id = newId();
+        int id = increaseAndGetCtr();
         // Prefixes system-generated variables with "#".
         return new VarIdentifier("#" + (systemVarCounter++), id);
     }
 
-    public void markCounter() {
-        mark = varCounter;
-    }
-
-    public void resetCounter() {
-        varCounter = mark;
-    }
-
-    private int newId() {
-        varCounter++;
-        return varCounter;
+    private int increaseAndGetCtr() {
+        varCounter.inc();
+        return varCounter.get();
     }
 
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
index d2cc431..d35516b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CompactStatement implements Statement {
+public class CompactStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
index b0a3f6e..eaa95cd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
@@ -22,13 +22,14 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class ConnectFeedStatement implements Statement {
+public class ConnectFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
index dcbf334..9881f83 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
@@ -19,12 +19,13 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
-public class CreateDataverseStatement implements Statement {
+public class CreateDataverseStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private String format;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
index 2deec27..ea6955b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
@@ -21,10 +21,11 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CreateFeedPolicyStatement implements Statement {
+public class CreateFeedPolicyStatement extends AbstractStatement {
 
     private final String policyName;
     private final String sourcePolicyName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index a4acfdf..5e82a38 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -35,7 +36,7 @@
  * The new create feed statement only concerns the feed adaptor configuration.
  * All feeds are considered as primary feeds.
  */
-public class CreateFeedStatement implements Statement {
+public class CreateFeedStatement extends AbstractStatement {
 
     private final Pair<Identifier, Identifier> qName;
     private final boolean ifNotExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
index c5fa524..f1cc6ba 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
@@ -23,12 +23,13 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CreateFunctionStatement implements Statement {
+public class CreateFunctionStatement extends AbstractStatement {
 
     private final FunctionSignature signature;
     private final String functionBody;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 1dc8673..2e9f1f9 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -23,13 +23,14 @@
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class CreateIndexStatement implements Statement {
+public class CreateIndexStatement extends AbstractStatement {
 
     private Identifier indexName;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index f8cf62c..4aeb6d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -38,7 +39,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
-public class DatasetDecl implements Statement {
+public class DatasetDecl extends AbstractStatement {
     protected static final String[] WITH_OBJECT_FIELDS = new String[] { MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME };
     protected static final Set<String> WITH_OBJECT_FIELDS_SET = new HashSet<>(Arrays.asList(WITH_OBJECT_FIELDS));
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
index 1b0c5cd..99303ce 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DataverseDecl implements Statement {
+public class DataverseDecl extends AbstractStatement {
 
     private Identifier dataverseName;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
index 6349006..9e184c2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DataverseDropStatement implements Statement {
+public class DataverseDropStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
index 18045b5..f05efbc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
@@ -21,13 +21,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DeleteStatement implements Statement {
+public class DeleteStatement extends AbstractStatement {
 
     private VariableExpr vars;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
index 33fa559..cb33452 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
@@ -19,12 +19,13 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class DisconnectFeedStatement implements Statement {
+public class DisconnectFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
index 45f5875..1434620 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DropDatasetStatement implements Statement {
+public class DropDatasetStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
index 47e51e3..ef6c096 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FeedDropStatement implements Statement {
+public class FeedDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
index 1b23237..a95254b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FeedPolicyDropStatement implements Statement {
+public class FeedPolicyDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier policyName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
index 2ea6c40..4161824 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
@@ -22,12 +22,13 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FunctionDecl implements Statement {
+public class FunctionDecl extends AbstractStatement {
     private FunctionSignature signature;
     private List<VarIdentifier> paramList;
     private Expression funcBody;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
index 610a20d..0496ecc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
@@ -20,10 +20,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FunctionDropStatement implements Statement {
+public class FunctionDropStatement extends AbstractStatement {
 
     private final FunctionSignature signature;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
index c262855..39e1d31 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class IndexDropStatement implements Statement {
+public class IndexDropStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
index de12dee..20aea67 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
@@ -23,6 +23,7 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.Statement;
@@ -31,7 +32,7 @@
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class InsertStatement implements IReturningStatement {
+public class InsertStatement extends AbstractStatement implements IReturningStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
index 92ea383..5366bc0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
@@ -21,11 +21,12 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LoadStatement implements Statement {
+public class LoadStatement extends AbstractStatement {
 
     private Identifier datasetName;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
index 5ec0181..7bb0c69 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class NodeGroupDropStatement implements Statement {
+public class NodeGroupDropStatement extends AbstractStatement {
 
     private Identifier nodeGroupName;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
index 1213f11..e4b3f5b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
@@ -21,11 +21,12 @@
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class NodegroupDecl implements Statement {
+public class NodegroupDecl extends AbstractStatement {
 
     private Identifier nodegroupName;
     private List<Identifier> nodeControllerNames;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
index 6f2b6de..6b532fe 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
@@ -23,13 +23,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class Query implements IReturningStatement {
+public class Query extends AbstractStatement implements IReturningStatement {
     private final boolean explain;
     private boolean topLevel = true;
     private Expression body;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
index 78d766c..4ca9d97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class RefreshExternalDatasetStatement implements Statement {
+public class RefreshExternalDatasetStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
index 17727af..8c52b45 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
@@ -19,10 +19,11 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class SetStatement implements Statement {
+public class SetStatement extends AbstractStatement {
 
     private String propName;
     private String propValue;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
index bc37c42..3ea791d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class StartFeedStatement implements Statement {
+public class StartFeedStatement extends AbstractStatement {
 
     public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
index 1fec6b2..99e5069 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
@@ -19,12 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class StopFeedStatement implements Statement {
+public class StopFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
index 4f0ea52..5430945 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
@@ -20,12 +20,13 @@
 
 import org.apache.asterix.common.annotations.TypeDataGen;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class TypeDecl implements Statement {
+public class TypeDecl extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier ident;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
index 059ac67..9f59f98 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class TypeDropStatement implements Statement {
+public class TypeDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private Identifier typeName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
index 0bdbbd1..e631107 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
@@ -22,13 +22,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.UpdateClause;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UpdateStatement implements Statement {
+public class UpdateStatement extends AbstractStatement {
 
     private VariableExpr vars;
     private Expression target;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
index 5dd9935..aabd999 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class WriteStatement implements Statement {
+public class WriteStatement extends AbstractStatement {
 
     private final Identifier ncName;
     private final String fileName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
index e11f169..f4f2ae1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
@@ -31,11 +31,10 @@
     MINUS("-"),
     CONCAT("||"),
     MUL("*"),
-    DIV("/"), // float/double
-    // divide
-    MOD("%"),
+    DIVIDE("/"),
+    DIV("div"),
+    MOD("mod"),
     CARET("^"),
-    IDIV("idiv"), // integer divide
     FUZZY_EQ("~="),
     LIKE("like"),
     NOT_LIKE("not_like"),
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
index 3e6caab..372acab 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
@@ -39,7 +39,6 @@
         addFunctionMapping("upper", "uppercase"); // upper, internal: uppercase
         addFunctionMapping("title", "initcap"); // title, internal: initcap
         addFunctionMapping("regexp_contains", "matches"); // regexp_contains, internal: matches
-        addFunctionMapping("power", "caret"); //pow, internal: caret
         addFunctionMapping("int", "integer"); // int, internal: integer
 
         // The "mapped-to" names are to be deprecated.
@@ -74,6 +73,11 @@
         addFunctionMapping("ifinf", "if-inf"); // ifinf, internal: if-inf
         addFunctionMapping("ifnan", "if-nan"); // ifnan, internal: if-nan
         addFunctionMapping("ifnanorinf", "if-nan-or-inf"); // ifnanorinf, internal: if-nan-or-inf
+        addFunctionMapping("missingif", "missing-if"); // missingif, internal: missing-if
+        addFunctionMapping("nanif", "nan-if"); // nanif, internal: nan-if
+        addFunctionMapping("neginfif", "neginf-if"); // neginfif, internal: neginf-if
+        addFunctionMapping("nullif", "null-if"); // nullif, internal: null-if
+        addFunctionMapping("posinfif", "posinf-if"); // posinfif, internal: posinf-if
         addFunctionMapping("toarray", "to-array"); // toarray, internal: to-array
         addFunctionMapping("toatomic", "to-atomic"); // toatomic, internal: to-atomic
         addFunctionMapping("toatom", "to-atomic"); // toatom, internal: to-atomic
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 3571dcf..1f564a5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -25,6 +25,7 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
@@ -43,6 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FunctionUtil {
 
@@ -68,7 +70,8 @@
 
     @FunctionalInterface
     public interface IFunctionNormalizer {
-        FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs) throws CompilationException;
+        FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, SourceLocation sourceLoc)
+                throws CompilationException;
     }
 
     /**
@@ -116,8 +119,9 @@
                 if (!namespace.equals(FunctionConstants.ASTERIX_NS)
                         && !namespace.equals(AlgebricksBuiltinFunctions.ALGEBRICKS_NS)
                         && metadataProvider.findDataverse(namespace) == null) {
-                    throw new CompilationException("In function call \"" + namespace + "." + signature.getName()
-                            + "(...)\", the dataverse \"" + namespace + "\" cannot be found!");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
+                            "In function call \"" + namespace + "." + signature.getName() + "(...)\", the dataverse \""
+                                    + namespace + "\" cannot be found!");
                 }
             } catch (AlgebricksException e) {
                 throw new CompilationException(e);
@@ -130,7 +134,8 @@
             }
             if (function == null) {
                 FunctionSignature normalizedSignature = functionNormalizer == null ? signature
-                        : functionNormalizer.normalizeBuiltinFunctionSignature(signature);
+                        : functionNormalizer.normalizeBuiltinFunctionSignature(signature,
+                                functionCall.getSourceLocation());
                 if (BuiltinFunctions.isBuiltinCompilerFunction(normalizedSignature, includePrivateFunctions)) {
                     continue;
                 }
@@ -141,7 +146,8 @@
                 } else {
                     messageBuilder.append("function " + signature + " is not defined");
                 }
-                throw new CompilationException(messageBuilder.toString());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
+                        messageBuilder.toString());
             }
 
             if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
@@ -149,7 +155,7 @@
                 FunctionDecl functionDecl = functionParser.getFunctionDecl(function);
                 if (functionDecl != null) {
                     if (functionDecls.contains(functionDecl)) {
-                        throw new CompilationException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
                                 "Recursive invocation " + functionDecls.get(functionDecls.size() - 1).getSignature()
                                         + " <==> " + functionDecl.getSignature());
                     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index c4b23ef..8367fa0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -213,6 +213,8 @@
                 return aOrderedListToString((AOrderedList) aObj);
             case STRING:
                 return ((AString) aObj).getStringValue();
+            case BOOLEAN:
+                return ((ABoolean) aObj).getBoolean().toString();
             default:
                 throw new AlgebricksException("value of type " + aObj.getType() + " is not supported yet");
         }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index 8ae67d6..c43824d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -60,6 +60,7 @@
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, List<FunctionDecl>> {
 
@@ -289,12 +290,18 @@
                 if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
                     subts.addSubstituion(new VariableExpr(param), e);
                 } else {
+                    SourceLocation sourceLoc = e.getSourceLocation();
                     VarIdentifier newV = context.newVariable();
                     Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
                             e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
-                    LetClause c = new LetClause(new VariableExpr(newV), (Expression) p1.first);
+                    VariableExpr newVRef1 = new VariableExpr(newV);
+                    newVRef1.setSourceLocation(sourceLoc);
+                    LetClause c = new LetClause(newVRef1, (Expression) p1.first);
+                    c.setSourceLocation(sourceLoc);
                     clauses.add(c);
-                    subts.addSubstituion(new VariableExpr(param), new VariableExpr(newV));
+                    VariableExpr newVRef2 = new VariableExpr(newV);
+                    newVRef2.setSourceLocation(sourceLoc);
+                    subts.addSubstituion(new VariableExpr(param), newVRef2);
                 }
             }
 
@@ -325,7 +332,9 @@
     }
 
     private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
+        SourceLocation sourceLoc = fnDecl.getSourceLocation();
         Query wrappedQuery = new Query(false);
+        wrappedQuery.setSourceLocation(sourceLoc);
         wrappedQuery.setBody(fnDecl.getFuncBody());
         wrappedQuery.setTopLevel(false);
         wrappedQuery.setExternalVars(fnDecl.getParamList());
@@ -340,7 +349,7 @@
             try {
                 fnDataverse = metadataProvider.findDataverse(fnNamespace);
             } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.NO_DATAVERSE_WITH_NAME, e, fnNamespace);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnNamespace);
             }
         }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index 58c81a6..64023a8 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -25,6 +25,7 @@
 import java.util.Map.Entry;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -73,6 +74,7 @@
         VariableExpr varExpr = lc.getVarExpr();
         VariableExpr newVe = generateNewVariable(context, varExpr);
         LetClause newLet = new LetClause(newVe, (Expression) p1.first);
+        newLet.setSourceLocation(lc.getSourceLocation());
         return new Pair<>(newLet, VariableCloneAndSubstitutionUtil.eliminateSubstFromList(lc.getVarExpr(), env));
     }
 
@@ -107,6 +109,7 @@
         }
         GroupbyClause newGroup = new GroupbyClause(newGbyList, newDecorList, newWithMap, newGroupVar, newGroupFieldList,
                 gc.hasHashGroupByHint(), gc.isGroupAll());
+        newGroup.setSourceLocation(gc.getSourceLocation());
         return new Pair<>(newGroup, newSubs);
     }
 
@@ -126,6 +129,7 @@
         }
         Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = qe.getSatisfiesExpr().accept(this, newSubs);
         QuantifiedExpression qe2 = new QuantifiedExpression(qe.getQuantifier(), newPairs, (Expression) p2.first);
+        qe2.setSourceLocation(qe.getSourceLocation());
         return new Pair<>(qe2, newSubs);
     }
 
@@ -134,6 +138,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = wc.getWhereExpr().accept(this, env);
         WhereClause newW = new WhereClause((Expression) p1.first);
+        newW.setSourceLocation(wc.getSourceLocation());
         return new Pair<>(newW, p1.second);
     }
 
@@ -142,6 +147,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         List<Expression> exprList = VariableCloneAndSubstitutionUtil.visitAndCloneExprList(pf.getExprList(), env, this);
         CallExpr f = new CallExpr(pf.getFunctionSignature(), exprList);
+        f.setSourceLocation(pf.getSourceLocation());
         return new Pair<>(f, env);
     }
 
@@ -152,11 +158,12 @@
         for (VarIdentifier vi : fd.getParamList()) {
             VariableExpr varExpr = new VariableExpr(vi);
             if (!env.constainsOldVar(varExpr)) {
-                throw new CompilationException("Parameter " + vi + " does not appear in the substitution list.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fd.getSourceLocation(),
+                        "Parameter " + vi + " does not appear in the substitution list.");
             }
             Expression newExpr = env.findSubstitution(varExpr);
             if (newExpr.getKind() != Kind.VARIABLE_EXPRESSION) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fd.getSourceLocation(),
                         "Parameter " + vi + " cannot be substituted by a non-variable expression.");
             }
             newList.add(((VariableExpr) newExpr).getVar());
@@ -164,6 +171,7 @@
 
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = fd.getFuncBody().accept(this, env);
         FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first);
+        newF.setSourceLocation(fd.getSourceLocation());
         return new Pair<>(newF, env);
     }
 
@@ -174,6 +182,7 @@
         Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = ifexpr.getThenExpr().accept(this, env);
         Pair<ILangExpression, VariableSubstitutionEnvironment> p3 = ifexpr.getElseExpr().accept(this, env);
         IfExpr i = new IfExpr((Expression) p1.first, (Expression) p2.first, (Expression) p3.first);
+        i.setSourceLocation(ifexpr.getSourceLocation());
         return new Pair<>(i, env);
     }
 
@@ -189,6 +198,7 @@
             p2 = new Pair<>(null, null);
         }
         LimitClause c = new LimitClause((Expression) p1.first, (Expression) p2.first);
+        c.setSourceLocation(lc.getSourceLocation());
         return new Pair<>(c, env);
     }
 
@@ -198,6 +208,7 @@
         List<Expression> oldExprList = lc.getExprList();
         List<Expression> exprs = VariableCloneAndSubstitutionUtil.visitAndCloneExprList(oldExprList, env, this);
         ListConstructor c = new ListConstructor(lc.getType(), exprs);
+        c.setSourceLocation(lc.getSourceLocation());
         return new Pair<>(c, env);
     }
 
@@ -217,6 +228,7 @@
             exprs.add((Expression) p1.first);
         }
         OperatorExpr oe = new OperatorExpr(exprs, op.getExprBroadcastIdx(), op.getOpList(), op.isCurrentop());
+        oe.setSourceLocation(op.getSourceLocation());
         return new Pair<>(oe, env);
     }
 
@@ -229,6 +241,7 @@
         oc2.setNumFrames(oc.getNumFrames());
         oc2.setNumTuples(oc.getNumTuples());
         oc2.setRangeMap(oc.getRangeMap());
+        oc2.setSourceLocation(oc.getSourceLocation());
         return new Pair<>(oc2, env);
     }
 
@@ -238,6 +251,7 @@
         Query newQ = new Query(q.isExplain());
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = q.getBody().accept(this, env);
         newQ.setBody((Expression) p1.first);
+        newQ.setSourceLocation(q.getSourceLocation());
         return new Pair<>(newQ, p1.second);
     }
 
@@ -253,6 +267,7 @@
             newFbs.add(fb2);
         }
         RecordConstructor newRc = new RecordConstructor(newFbs);
+        newRc.setSourceLocation(rc.getSourceLocation());
         return new Pair<>(newRc, env);
     }
 
@@ -261,6 +276,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = u.getExpr().accept(this, env);
         UnaryExpr newU = new UnaryExpr(u.getExprType(), (Expression) p1.first);
+        newU.setSourceLocation(u.getSourceLocation());
         return new Pair<>(newU, env);
     }
 
@@ -275,6 +291,7 @@
         }
         IndexAccessor i = new IndexAccessor((Expression) p1.first, indexExpr);
         i.setAny(ia.isAny());
+        i.setSourceLocation(ia.getSourceLocation());
         return new Pair<>(i, env);
     }
 
@@ -283,6 +300,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p = fa.getExpr().accept(this, env);
         FieldAccessor newF = new FieldAccessor((Expression) p.first, fa.getIdent());
+        newF.setSourceLocation(fa.getSourceLocation());
         return new Pair<>(newF, p.second);
     }
 
@@ -301,7 +319,9 @@
             // it is a variable from the context
             VarIdentifier var = context.getRewrittenVar(expr.getVar().getId());
             if (var != null) {
-                return new VariableExpr(var);
+                VariableExpr newVarExpr = new VariableExpr(var);
+                newVarExpr.setSourceLocation(expr.getSourceLocation());
+                return newVarExpr;
             }
         }
         return expr;
@@ -319,7 +339,9 @@
     public VariableExpr generateNewVariable(LangRewritingContext context, VariableExpr varExpr) {
         VarIdentifier vi = varExpr.getVar();
         VarIdentifier newVar = context.mapOldId(vi.getId(), vi.getValue());
-        return new VariableExpr(newVar);
+        VariableExpr newVarExpr = new VariableExpr(newVar);
+        newVarExpr.setSourceLocation(varExpr.getSourceLocation());
+        return newVarExpr;
     }
 
     /**
diff --git a/asterixdb/asterix-lang-sqlpp/pom.xml b/asterixdb/asterix-lang-sqlpp/pom.xml
index 0321437..e122335 100644
--- a/asterixdb/asterix-lang-sqlpp/pom.xml
+++ b/asterixdb/asterix-lang-sqlpp/pom.xml
@@ -162,6 +162,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-core</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
index 40a5ea5..ba1c498 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
@@ -21,12 +21,12 @@
 
 import java.util.Objects;
 
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
 
-public abstract class AbstractBinaryCorrelateClause implements Clause {
+public abstract class AbstractBinaryCorrelateClause extends AbstractClause {
 
     private JoinType joinType;
     private Expression rightExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
index 65c2f01..4b0e324 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
@@ -19,18 +19,17 @@
 
 package org.apache.asterix.lang.sqlpp.clause;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class FromClause implements Clause {
+public class FromClause extends AbstractClause {
 
-    private List<FromTerm> fromTerms = new ArrayList<>();
+    private List<FromTerm> fromTerms;
 
     public FromClause(List<FromTerm> fromTerms) {
         this.fromTerms = fromTerms;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
index 6860cfe..3e26371 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
@@ -24,13 +24,13 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class FromTerm implements Clause {
+public class FromTerm extends AbstractClause {
     private Expression leftExpr;
     private VariableExpr leftVar;
     private VariableExpr posVar;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
index aadbe42..2c0ce0c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
@@ -19,12 +19,12 @@
 package org.apache.asterix.lang.sqlpp.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class HavingClause implements Clause {
+public class HavingClause extends AbstractClause {
 
     private Expression filterExpression;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
index 066ad22..7f353d6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
@@ -22,12 +22,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class Projection implements Clause {
+public class Projection extends AbstractClause {
 
     private Expression expr;
     private String name;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
index 0477b1c..f072fe4 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
@@ -24,14 +24,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.WhereClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectBlock implements Clause {
+public class SelectBlock extends AbstractClause {
 
     private SelectClause selectClause;
     private FromClause fromClause;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
index e5c3f56..2627870 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
@@ -22,11 +22,11 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectClause implements Clause {
+public class SelectClause extends AbstractClause {
 
     private SelectElement selectElement;
     private SelectRegular selectRegular;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
index 9c82d5c..464b744 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.lang.sqlpp.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectElement implements Clause {
+public class SelectElement extends AbstractClause {
 
     private Expression expr;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
index a114337..9a191723 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
@@ -23,11 +23,11 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectRegular implements Clause {
+public class SelectRegular extends AbstractClause {
 
     private List<Projection> projections;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
index 18d9879..7994358 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
@@ -24,13 +24,13 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectSetOperation implements Clause {
+public class SelectSetOperation extends AbstractClause {
 
     private SetOperationInput leftInput;
     private List<SetOperationRight> rightInputs = new ArrayList<>();
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java
new file mode 100644
index 0000000..52752bb
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.sqlpp.parser;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public final class SqlppParseException extends ParseException {
+
+    private final SourceLocation sourceLoc;
+
+    public SqlppParseException(SourceLocation sourceLoc, String message) {
+        super(message);
+        this.sourceLoc = sourceLoc;
+    }
+
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index befa5ab..647b50e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -139,7 +139,7 @@
         rewriteDistinctAggregations();
 
         // Sets the var counter of the query.
-        topStatement.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter().get());
     }
 
     protected void rewriteGroupByAggregationSugar() throws CompilationException {
@@ -225,7 +225,8 @@
         for (Expression topLevelExpr : topExpr.getDirectlyEnclosedExpressions()) {
             usedStoredFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr,
                     funIds, null, expr -> getFunctionCalls(expr), func -> functionRepository.getFunctionDecl(func),
-                    signature -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false)));
+                    (signature, sourceLoc) -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false,
+                            sourceLoc)));
         }
         declaredFunctions.addAll(usedStoredFunctionDecls);
         if (inlineUdfs && !declaredFunctions.isEmpty()) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
index b945a40..14a7ec0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
@@ -57,7 +57,9 @@
     public Expression visit(GroupbyClause groupbyClause, ILangExpression arg) throws CompilationException {
         for (GbyVariableExpressionPair gbyKeyPair : groupbyClause.getGbyPairList()) {
             if (gbyKeyPair.getVar() == null) {
-                gbyKeyPair.setVar(new VariableExpr(context.newVariable()));
+                VariableExpr varExpr = new VariableExpr(context.newVariable());
+                varExpr.setSourceLocation(gbyKeyPair.getExpr().getSourceLocation());
+                gbyKeyPair.setVar(varExpr);
             }
         }
         return super.visit(groupbyClause, arg);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 7e228f4..f29dbf1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVisitor {
 
@@ -76,14 +77,17 @@
     }
 
     private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) {
-        Expression likeExpr =
+        CallExpr likeExpr =
                 new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
+        likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
         switch (opType) {
             case LIKE:
                 return likeExpr;
             case NOT_LIKE:
-                return new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+                CallExpr notLikeExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
                         new ArrayList<>(Collections.singletonList(likeExpr)));
+                notLikeExpr.setSourceLocation(operatorExpr.getSourceLocation());
+                return notLikeExpr;
             default:
                 throw new IllegalArgumentException(String.valueOf(opType));
         }
@@ -91,29 +95,37 @@
 
     private Expression processInOperator(OperatorExpr operatorExpr, OperatorType opType) throws CompilationException {
         VariableExpr bindingVar = new VariableExpr(context.newVariable());
+        bindingVar.setSourceLocation(operatorExpr.getSourceLocation());
         Expression itemExpr = operatorExpr.getExprList().get(0);
         Expression collectionExpr = operatorExpr.getExprList().get(1);
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(itemExpr);
         comparison.addOperand(bindingVar);
         comparison.setCurrentop(true);
+        comparison.setSourceLocation(operatorExpr.getSourceLocation());
         if (opType == OperatorType.IN) {
             comparison.addOperator(OperatorType.EQ);
-            return new QuantifiedExpression(Quantifier.SOME,
+            QuantifiedExpression quantExpr = new QuantifiedExpression(Quantifier.SOME,
                     new ArrayList<>(Collections.singletonList(new QuantifiedPair(bindingVar, collectionExpr))),
                     comparison);
+            quantExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return quantExpr;
         } else {
             comparison.addOperator(OperatorType.NEQ);
-            return new QuantifiedExpression(Quantifier.EVERY,
+            QuantifiedExpression quantExpr = new QuantifiedExpression(Quantifier.EVERY,
                     new ArrayList<>(Collections.singletonList(new QuantifiedPair(bindingVar, collectionExpr))),
                     comparison);
+            quantExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return quantExpr;
         }
     }
 
     private Expression processConcatOperator(OperatorExpr operatorExpr) {
         // All operators have to be "||"s (according to the grammar).
-        return new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, FunctionMapUtil.CONCAT, 1),
+        CallExpr callExpr = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, FunctionMapUtil.CONCAT, 1),
                 operatorExpr.getExprList());
+        callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+        return callExpr;
     }
 
     private Expression processBetweenOperator(OperatorExpr operatorExpr, OperatorType opType)
@@ -124,24 +136,33 @@
         Expression right = operatorExpr.getExprList().get(2);
 
         // Creates the expression left <= target.
-        Expression leftComparison = createLessThanExpression(left, target, operatorExpr.getHints());
+        Expression leftComparison =
+                createLessThanExpression(left, target, operatorExpr.getHints(), operatorExpr.getSourceLocation());
         // Creates the expression target <= right.
-        Expression rightComparison = createLessThanExpression(target, right, operatorExpr.getHints());
+        Expression rightComparison =
+                createLessThanExpression(target, right, operatorExpr.getHints(), operatorExpr.getSourceLocation());
         OperatorExpr andExpr = new OperatorExpr();
         andExpr.addOperand(leftComparison);
         andExpr.addOperand(rightComparison);
         andExpr.addOperator(OperatorType.AND);
-        return opType == OperatorType.BETWEEN ? andExpr
-                : new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
-                        new ArrayList<>(Collections.singletonList(andExpr)));
+        andExpr.setSourceLocation(operatorExpr.getSourceLocation());
+        if (opType == OperatorType.BETWEEN) {
+            return andExpr;
+        } else {
+            CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+                    new ArrayList<>(Collections.singletonList(andExpr)));
+            callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return callExpr;
+        }
     }
 
-    private Expression createLessThanExpression(Expression lhs, Expression rhs, List<IExpressionAnnotation> hints)
-            throws CompilationException {
+    private Expression createLessThanExpression(Expression lhs, Expression rhs, List<IExpressionAnnotation> hints,
+            SourceLocation sourceLoc) {
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(lhs);
         comparison.addOperand(rhs);
         comparison.addOperator(OperatorType.LE);
+        comparison.setSourceLocation(sourceLoc);
         if (hints != null) {
             for (IExpressionAnnotation hint : hints) {
                 comparison.addHint(hint);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
index ddf3cd6..f0a0f87 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This visitor rewrites set operation queries with order by and limit into
@@ -81,19 +82,28 @@
         OrderbyClause orderBy = selectExpression.getOrderbyClause();
         LimitClause limit = selectExpression.getLimitClause();
 
+        SourceLocation sourceLoc = selectExpression.getSourceLocation();
+
         // Wraps the set operation part with a subquery.
         SelectExpression nestedSelectExpression = new SelectExpression(null, selectSetOperation, null, null, true);
+        nestedSelectExpression.setSourceLocation(sourceLoc);
         VariableExpr newBindingVar = new VariableExpr(context.newVariable()); // Binding variable for the subquery.
+        newBindingVar.setSourceLocation(sourceLoc);
         FromTerm newFromTerm = new FromTerm(nestedSelectExpression, newBindingVar, null, null);
+        newFromTerm.setSourceLocation(sourceLoc);
         FromClause newFromClause = new FromClause(new ArrayList<>(Collections.singletonList(newFromTerm)));
+        newFromClause.setSourceLocation(sourceLoc);
         SelectClause selectClause = new SelectClause(new SelectElement(newBindingVar), null, false);
+        selectClause.setSourceLocation(sourceLoc);
         SelectBlock selectBlock = new SelectBlock(selectClause, newFromClause, null, null, null, null, null);
+        selectBlock.setSourceLocation(sourceLoc);
         SelectSetOperation newSelectSetOperation =
                 new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-
+        newSelectSetOperation.setSourceLocation(sourceLoc);
         // Puts together the generated select-from-where query and order by/limit.
         SelectExpression newSelectExpression = new SelectExpression(selectExpression.getLetList(),
                 newSelectSetOperation, orderBy, limit, selectExpression.isSubquery());
+        newSelectExpression.setSourceLocation(sourceLoc);
         return super.visit(newSelectExpression, arg);
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
index 8d59a3b..abfa14e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
@@ -42,7 +42,8 @@
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         //TODO(buyingyi): rewrite SQL temporal functions
         FunctionSignature functionSignature = callExpr.getFunctionSignature();
-        callExpr.setFunctionSignature(FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, true));
+        callExpr.setFunctionSignature(FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, true,
+                callExpr.getSourceLocation()));
         List<Expression> newExprList = new ArrayList<>();
         for (Expression expr : callExpr.getExprList()) {
             newExprList.add(expr.accept(this, arg));
@@ -73,13 +74,16 @@
             newExprList.add(thenExprList.get(index));
         }
         newExprList.add(newCaseExpr.getElseExpr());
-        return new CallExpr(functionSignature, newExprList);
+        CallExpr callExpr = new CallExpr(functionSignature, newExprList);
+        callExpr.setSourceLocation(caseExpr.getSourceLocation());
+        return callExpr;
     }
 
     // Normalizes WHEN expressions so that it can have correct NULL/MISSING semantics as well
     // as type promotion semantics.
     private CaseExpression normalizeCaseExpr(CaseExpression caseExpr) throws CompilationException {
         LiteralExpr trueLiteral = new LiteralExpr(TrueLiteral.INSTANCE);
+        trueLiteral.setSourceLocation(caseExpr.getSourceLocation());
         Expression conditionExpr = caseExpr.getConditionExpr();
         if (trueLiteral.equals(conditionExpr)) {
             return caseExpr;
@@ -90,9 +94,13 @@
             operatorExpr.addOperand((Expression) SqlppRewriteUtil.deepCopy(expr));
             operatorExpr.addOperand(caseExpr.getConditionExpr());
             operatorExpr.addOperator(OperatorType.EQ);
+            operatorExpr.setSourceLocation(expr.getSourceLocation());
             normalizedWhenExprs.add(operatorExpr);
         }
-        return new CaseExpression(trueLiteral, normalizedWhenExprs, caseExpr.getThenExprs(), caseExpr.getElseExpr());
+        CaseExpression newCaseExpr =
+                new CaseExpression(trueLiteral, normalizedWhenExprs, caseExpr.getThenExprs(), caseExpr.getElseExpr());
+        newCaseExpr.setSourceLocation(caseExpr.getSourceLocation());
+        return newCaseExpr;
     }
 
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
index fd50c8e..80844ab 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * An AST pre-processor to rewrite distinct aggregates into regular aggregates as follows: <br/>
@@ -81,17 +82,28 @@
      * rewrites {@code expr -> FROM expr AS i SELECT DISTINCT VALUE i}
      */
     private Expression rewriteArgument(Expression argExpr) throws CompilationException {
+        SourceLocation sourceLoc = argExpr.getSourceLocation();
         // From clause
         VariableExpr fromBindingVar = new VariableExpr(context.newVariable());
+        fromBindingVar.setSourceLocation(sourceLoc);
         FromTerm fromTerm = new FromTerm(argExpr, fromBindingVar, null, null);
+        fromTerm.setSourceLocation(sourceLoc);
         FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+        fromClause.setSourceLocation(sourceLoc);
 
         // Select clause.
-        SelectClause selectClause = new SelectClause(new SelectElement(fromBindingVar), null, true);
+        SelectElement selectElement = new SelectElement(fromBindingVar);
+        selectElement.setSourceLocation(sourceLoc);
+        SelectClause selectClause = new SelectClause(selectElement, null, true);
+        selectClause.setSourceLocation(sourceLoc);
 
         // Construct the select expression.
         SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, null, null, null, null);
+        selectBlock.setSourceLocation(sourceLoc);
         SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-        return new SelectExpression(null, selectSetOperation, null, null, true);
+        selectSetOperation.setSourceLocation(sourceLoc);
+        SelectExpression selectExpr = new SelectExpression(null, selectSetOperation, null, null, true);
+        selectExpr.setSourceLocation(sourceLoc);
+        return selectExpr;
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
index c5f0a54..e133bad 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -48,6 +49,7 @@
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -259,12 +261,16 @@
         }
 
         private Expression wrapAggregationArgument(Expression argExpr) throws CompilationException {
+            SourceLocation sourceLoc = argExpr.getSourceLocation();
             Expression expr = argExpr;
             Set<VariableExpr> freeVars = SqlppRewriteUtil.getFreeVariable(expr);
 
             VariableExpr fromBindingVar = new VariableExpr(context.newVariable());
+            fromBindingVar.setSourceLocation(sourceLoc);
             FromTerm fromTerm = new FromTerm(groupVar, fromBindingVar, null, null);
+            fromTerm.setSourceLocation(sourceLoc);
             FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+            fromClause.setSourceLocation(sourceLoc);
 
             // Maps field variable expressions to field accesses.
             Map<Expression, Expression> varExprMap = new HashMap<>();
@@ -272,31 +278,42 @@
                 // Reference to a field in the group variable.
                 if (fieldVars.containsKey(usedVar)) {
                     // Rewrites to a reference to a field in the group variable.
-                    varExprMap.put(usedVar,
-                            new FieldAccessor(fromBindingVar, new VarIdentifier(fieldVars.get(usedVar).getValue())));
+                    FieldAccessor fa =
+                            new FieldAccessor(fromBindingVar, new VarIdentifier(fieldVars.get(usedVar).getValue()));
+                    fa.setSourceLocation(usedVar.getSourceLocation());
+                    varExprMap.put(usedVar, fa);
                 } else if (outerVars.contains(usedVar)) {
                     // Do nothing
                 } else if (fieldVars.size() == 1) {
                     // Rewrites to a reference to a single field in the group variable.
-                    varExprMap.put(usedVar,
-                            new FieldAccessor(new FieldAccessor(fromBindingVar, fieldVars.values().iterator().next()),
-                                    SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar())));
+                    FieldAccessor faInner = new FieldAccessor(fromBindingVar, fieldVars.values().iterator().next());
+                    faInner.setSourceLocation(usedVar.getSourceLocation());
+                    FieldAccessor faOuter =
+                            new FieldAccessor(faInner, SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar()));
+                    faOuter.setSourceLocation(usedVar.getSourceLocation());
+                    varExprMap.put(usedVar, faOuter);
                 } else {
-                    throw new CompilationException("Cannot resolve alias reference for undefined identifier "
-                            + usedVar.getVar().getValue() + " in " + fieldVars);
+                    throw new CompilationException(ErrorCode.UNDEFINED_IDENTIFIER, usedVar.getSourceLocation(),
+                            usedVar.getVar().getValue(), String.valueOf(fieldVars));
                 }
             }
 
             // Select clause.
             SelectElement selectElement =
                     new SelectElement(SqlppRewriteUtil.substituteExpression(expr, varExprMap, context));
+            selectElement.setSourceLocation(sourceLoc);
             SelectClause selectClause = new SelectClause(selectElement, null, false);
+            selectClause.setSourceLocation(sourceLoc);
 
             // Construct the select expression.
             SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, null, null, null, null);
+            selectBlock.setSourceLocation(sourceLoc);
             SelectSetOperation selectSetOperation =
                     new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-            return new SelectExpression(null, selectSetOperation, null, null, true);
+            selectSetOperation.setSourceLocation(sourceLoc);
+            SelectExpression selectExpr = new SelectExpression(null, selectSetOperation, null, null, true);
+            selectExpr.setSourceLocation(sourceLoc);
+            return selectExpr;
         }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
index bee5830..faca85e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
@@ -72,6 +72,7 @@
         // Sets the group variable.
         if (!gbyClause.hasGroupVar()) {
             VariableExpr groupVar = new VariableExpr(context.newVariable());
+            groupVar.setSourceLocation(gbyClause.getSourceLocation());
             gbyClause.setGroupVar(groupVar);
         }
 
@@ -95,9 +96,11 @@
             List<GbyVariableExpressionPair> gbyPairList = new ArrayList<>();
             List<GbyVariableExpressionPair> decorPairList = new ArrayList<>();
             VariableExpr groupVar = new VariableExpr(context.newVariable());
+            groupVar.setSourceLocation(selectBlock.getSourceLocation());
             List<Pair<Expression, Identifier>> groupFieldList = createGroupFieldList(selectBlock);
             GroupbyClause gbyClause = new GroupbyClause(gbyPairList, decorPairList, new HashMap<>(), groupVar,
                     groupFieldList, false, true);
+            gbyClause.setSourceLocation(selectBlock.getSourceLocation());
             selectBlock.setGroupbyClause(gbyClause);
         }
     }
@@ -128,8 +131,10 @@
     private void addToGroupFieldList(List<Pair<Expression, Identifier>> groupFieldList,
             Collection<VariableExpr> fromBindingVars) {
         for (VariableExpr varExpr : fromBindingVars) {
-            Pair<Expression, Identifier> varIdPair = new Pair<>(new VariableExpr(varExpr.getVar()),
-                    SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar()));
+            VariableExpr newVarExpr = new VariableExpr(varExpr.getVar());
+            newVarExpr.setSourceLocation(varExpr.getSourceLocation());
+            Pair<Expression, Identifier> varIdPair =
+                    new Pair<>(newVarExpr, SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar()));
             groupFieldList.add(varIdPair);
         }
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 04f8bc6..5af284b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -23,6 +23,7 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
@@ -43,6 +44,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopingVisitor {
 
@@ -94,9 +96,10 @@
     private Expression resolve(VariableExpr varExpr, String dataverseName, String datasetName,
             Expression originalExprWithUndefinedIdentifier, ILangExpression parent) throws CompilationException {
 
-        String varName = varExpr.getVar().getValue();
+        SourceLocation sourceLoc = varExpr.getSourceLocation();
 
-        VarIdentifier var = lookupVariable(varName);
+        String varName = varExpr.getVar().getValue();
+        VarIdentifier var = lookupVariable(varName, sourceLoc);
         if (var != null) {
             // Exists such an identifier
             varExpr.setIsNewVar(false);
@@ -106,61 +109,63 @@
 
         boolean resolveToDatasetOnly = resolveToDatasetOnly(originalExprWithUndefinedIdentifier, parent);
         if (resolveToDatasetOnly) {
-            return resolveAsDataset(dataverseName, datasetName);
+            return resolveAsDataset(dataverseName, datasetName, sourceLoc);
         }
 
         Set<VariableExpr> localVars = scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
         switch (localVars.size()) {
             case 0:
-                return resolveAsDataset(dataverseName, datasetName);
+                return resolveAsDataset(dataverseName, datasetName, sourceLoc);
             case 1:
                 return resolveAsFieldAccess(localVars.iterator().next(),
-                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue());
+                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue(), sourceLoc);
             default:
                 // More than one possibilities.
-                throw new CompilationException("Cannot resolve ambiguous alias reference for undefined identifier "
-                        + SqlppVariableUtil.toUserDefinedVariableName(varName).getValue() + " in " + localVars);
+                throw new CompilationException(ErrorCode.AMBIGUOUS_IDENTIFIER, sourceLoc,
+                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue(), String.valueOf(localVars));
         }
     }
 
-    private VarIdentifier lookupVariable(String varName) throws CompilationException {
+    private VarIdentifier lookupVariable(String varName, SourceLocation sourceLoc) throws CompilationException {
         if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new CompilationException(
-                    "Inside limit clauses, it is disallowed to reference a variable having the same name"
-                            + " as any variable bound in the same scope as the limit clause.");
+            throw new CompilationException(ErrorCode.FORBIDDEN_SCOPE, sourceLoc);
         }
         Identifier ident = scopeChecker.lookupSymbol(varName);
         return ident != null ? (VarIdentifier) ident : null;
     }
 
-    private Expression resolveAsDataset(String dataverseName, String datasetName) throws CompilationException {
-        if (!datasetExists(dataverseName, datasetName)) {
-            throwUnresolvableError(dataverseName, datasetName);
+    private Expression resolveAsDataset(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
+        if (!datasetExists(dataverseName, datasetName, sourceLoc)) {
+            throwUnresolvableError(dataverseName, datasetName, sourceLoc);
         }
         String fullyQualifiedName = dataverseName == null ? datasetName : dataverseName + "." + datasetName;
         List<Expression> argList = new ArrayList<>(1);
         argList.add(new LiteralExpr(new StringLiteral(fullyQualifiedName)));
-        return new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
+        CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
     // Rewrites for an field access by name
-    private Expression resolveAsFieldAccess(VariableExpr var, String fieldName) throws CompilationException {
+    private Expression resolveAsFieldAccess(VariableExpr var, String fieldName, SourceLocation sourceLoc) {
         List<Expression> argList = new ArrayList<>(2);
         argList.add(var);
         argList.add(new LiteralExpr(new StringLiteral(fieldName)));
-        return new CallExpr(new FunctionSignature(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argList);
+        CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argList);
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
-    private void throwUnresolvableError(String dataverseName, String datasetName) throws CompilationException {
+    private void throwUnresolvableError(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
         String defaultDataverseName = metadataProvider.getDefaultDataverseName();
         if (dataverseName == null && defaultDataverseName == null) {
-            throw new CompilationException("Cannot find dataset " + datasetName
-                    + " because there is no dataverse declared, nor an alias with name " + datasetName + "!");
+            throw new CompilationException(ErrorCode.NAME_RESOLVE_UNKNOWN_DATASET, sourceLoc, datasetName);
         }
         //If no available dataset nor in-scope variable to resolve to, we throw an error.
-        throw new CompilationException("Cannot find dataset " + datasetName + " in dataverse "
-                + (dataverseName == null ? defaultDataverseName : dataverseName) + " nor an alias with name "
-                + datasetName + "!");
+        throw new CompilationException(ErrorCode.NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                dataverseName == null ? defaultDataverseName : dataverseName);
     }
 
     // For a From/Join/UNNEST/Quantifiers binding expression, we resolve the undefined identifier reference as
@@ -171,12 +176,13 @@
         return parent.accept(visitor, originalExpressionWithUndefinedIdentifier);
     }
 
-    private boolean datasetExists(String dataverseName, String datasetName) throws CompilationException {
+    private boolean datasetExists(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
         try {
             return metadataProvider.findDataset(dataverseName, datasetName) != null
                     || fullyQualifiedDatasetNameExists(datasetName);
         } catch (AlgebricksException e) {
-            throw new CompilationException(e);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
         }
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
index 96c61f2..c0eb2d9 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.sqlpp.parser.ParseException;
+import org.apache.asterix.lang.sqlpp.parser.SqlppParseException;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -44,13 +45,15 @@
             FieldAccessor fa = (FieldAccessor) expr;
             return SqlppVariableUtil.toInternalVariableName(fa.getIdent().getValue());
         } else {
+            String exprText;
             try {
-                throw new ParseException(
-                        "Need an alias for the enclosed expression:\n" + SqlppFormatPrintUtil.toString(expr));
+                exprText = SqlppFormatPrintUtil.toString(expr);
             } catch (CompilationException e) {
                 LOGGER.error(e.getLocalizedMessage(), e);
-                throw new ParseException(e.getLocalizedMessage());
+                throw new SqlppParseException(expr.getSourceLocation(), e.getLocalizedMessage());
             }
+            throw new SqlppParseException(expr.getSourceLocation(),
+                    "Need an alias for the enclosed expression:\n" + exprText);
         }
     }
 
@@ -93,6 +96,7 @@
             VarIdentifier var = new VarIdentifier(varName);
             VariableExpr varExpr = new VariableExpr();
             varExpr.setVar(var);
+            varExpr.setSourceLocation(expr.getSourceLocation());
             return varExpr;
         } catch (ParseException e) {
             if (raiseError) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
index 1ff7c9e..7773072 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -33,11 +34,14 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FunctionMapUtil {
 
     public static final String CONCAT = "concat";
-    private final static String CORE_AGGREGATE_PREFIX = "coll_";
+    private final static String CORE_AGGREGATE_PREFIX = "strict_";
+    // This is a transitional case. The ALT_CORE_AGGREGATE_PREFIX should be removed again.
+    private final static String ALT_CORE_AGGREGATE_PREFIX = "coll_";
     private final static String CORE_SQL_AGGREGATE_PREFIX = "array_";
     private final static String INTERNAL_SQL_AGGREGATE_PREFIX = "sql-";
 
@@ -69,31 +73,6 @@
     }
 
     /**
-     * Whether a function signature is a SQL++ core aggregate function.
-     *
-     * @param fs,
-     *            the function signature.
-     * @return true if the function signature is a SQL++ core aggregate,
-     *         false otherwise.
-     */
-    public static boolean isCoreAggregateFunction(FunctionSignature fs) {
-        String name = fs.getName().toLowerCase();
-        boolean coreAgg = name.startsWith(CORE_AGGREGATE_PREFIX);
-        boolean coreSqlAgg = name.startsWith(CORE_SQL_AGGREGATE_PREFIX);
-        if (!coreAgg && !coreSqlAgg) {
-            return false;
-        }
-        String internalName = coreAgg ? name.substring(CORE_AGGREGATE_PREFIX.length())
-                : (INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length()));
-        IFunctionInfo finfo = FunctionUtil
-                .getFunctionInfo(new FunctionIdentifier(FunctionConstants.ASTERIX_NS, internalName, fs.getArity()));
-        if (finfo == null) {
-            return false;
-        }
-        return BuiltinFunctions.getAggregateFunction(finfo.getFunctionIdentifier()) != null;
-    }
-
-    /**
      * Get the corresponding SQL++ core aggregate function from the SQL-92 aggregate function.
      *
      * @param fs,
@@ -112,18 +91,30 @@
     /**
      * Maps a user invoked function signature to a system internal function signature.
      *
-     * @param fs,
+     * @param fs
      *            the user typed function.
+     * @param checkSql92Aggregate
+     *            enable check if the function is a SQL-92 aggregate function
+     * @param sourceLoc
+     *            the source location of the function call
      * @return the system internal function.
+     * @throws CompilationException
+     *             if checkSql92Aggregate is true and the function is a SQL-92 aggregate function
      */
-    public static FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, boolean checkSql92Aggregate)
-            throws CompilationException {
-        if (isCoreAggregateFunction(fs)) {
-            return internalizeCoreAggregateFunctionName(fs);
+    public static FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, boolean checkSql92Aggregate,
+            SourceLocation sourceLoc) throws CompilationException {
+        String internalName = getInternalCoreAggregateFunctionName(fs);
+        if (internalName != null) {
+            FunctionIdentifier fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, internalName, fs.getArity());
+            IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
+            if (finfo != null && BuiltinFunctions.getAggregateFunction(finfo.getFunctionIdentifier()) != null) {
+                return new FunctionSignature(FunctionConstants.ASTERIX_NS, internalName, fs.getArity());
+            }
         } else if (checkSql92Aggregate && isSql92AggregateFunction(fs)) {
-            throw new CompilationException(fs.getName()
-                    + " is a SQL-92 aggregate function. The SQL++ core aggregate function " + CORE_SQL_AGGREGATE_PREFIX
-                    + fs.getName().toLowerCase() + " could potentially express the intent.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    fs.getName() + " is a SQL-92 aggregate function. The SQL++ core aggregate function "
+                            + CORE_SQL_AGGREGATE_PREFIX + fs.getName().toLowerCase()
+                            + " could potentially express the intent.");
         }
         String mappedName = CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(fs).getName();
         return new FunctionSignature(fs.getNamespace(), mappedName, fs.getArity());
@@ -143,26 +134,30 @@
             return callExpr;
         }
         callExpr.setFunctionSignature(new FunctionSignature(FunctionConstants.ASTERIX_NS, internalFuncName, 1));
-        callExpr.setExprList(new ArrayList<>(Collections.singletonList(
-                new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, callExpr.getExprList()))));
+        ListConstructor listConstr =
+                new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, callExpr.getExprList());
+        listConstr.setSourceLocation(callExpr.getSourceLocation());
+        callExpr.setExprList(new ArrayList<>(Collections.singletonList(listConstr)));
         return callExpr;
     }
 
     /**
-     * Removes the "array_" prefix for user-facing SQL++ core aggregate function names.
+     * Removes the "array_", "strict_", or "coll_" prefix for user-facing SQL++ core aggregate function names.
      *
-     * @param fs,
+     * @param fs
      *            a user-facing SQL++ core aggregate function signature.
-     * @return the AsterixDB internal function signature for the aggregate function.
-     * @throws CompilationException
+     * @return the AsterixDB internal function name for the aggregate function.
      */
-    private static FunctionSignature internalizeCoreAggregateFunctionName(FunctionSignature fs)
-            throws CompilationException {
+    private static String getInternalCoreAggregateFunctionName(FunctionSignature fs) {
         String name = fs.getName().toLowerCase();
-        boolean coreAgg = name.startsWith(CORE_AGGREGATE_PREFIX);
-        String lowerCaseName = coreAgg ? name.substring(CORE_AGGREGATE_PREFIX.length())
-                : (INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length()));
-        return new FunctionSignature(FunctionConstants.ASTERIX_NS, lowerCaseName, fs.getArity());
+        if (name.startsWith(CORE_AGGREGATE_PREFIX)) {
+            return name.substring(CORE_AGGREGATE_PREFIX.length());
+        } else if (name.startsWith(ALT_CORE_AGGREGATE_PREFIX)) {
+            return name.substring(ALT_CORE_AGGREGATE_PREFIX.length());
+        } else if (name.startsWith(CORE_SQL_AGGREGATE_PREFIX)) {
+            return INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length());
+        } else {
+            return null;
+        }
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
index 228ea23..5a30b42 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
@@ -83,6 +83,7 @@
         // Creates a wrapper query for the expression so that if the expression itself
         // is the key, it can also be replaced.
         Query wrapper = new Query(false);
+        wrapper.setSourceLocation(expression.getSourceLocation());
         wrapper.setBody(expression);
         // Creates a substitution visitor.
         SqlppSubstituteExpressionVisitor visitor = new SqlppSubstituteExpressionVisitor(context, exprMap);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 1166148..6635a0e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -78,7 +78,9 @@
         for (FromTerm fromTerm : fromClause.getFromTerms()) {
             fromTerms.add((FromTerm) fromTerm.accept(this, arg));
         }
-        return new FromClause(fromTerms);
+        FromClause copy = new FromClause(fromTerms);
+        copy.setSourceLocation(fromClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -94,7 +96,9 @@
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
             correlateClauses.add((AbstractBinaryCorrelateClause) correlateClause.accept(this, arg));
         }
-        return new FromTerm(fromExpr, fromVar, positionVar, correlateClauses);
+        FromTerm copy = new FromTerm(fromExpr, fromVar, positionVar, correlateClauses);
+        copy.setSourceLocation(fromTerm.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -104,8 +108,10 @@
         VariableExpr rightPositionVar = joinClause.getPositionalVariable() == null ? null
                 : (VariableExpr) joinClause.getPositionalVariable().accept(this, arg);
         Expression conditionExpresion = (Expression) joinClause.getConditionExpression().accept(this, arg);
-        return new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
+        JoinClause copy = new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
                 conditionExpresion);
+        copy.setSourceLocation(joinClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -115,8 +121,10 @@
         VariableExpr rightPositionVar = nestClause.getPositionalVariable() == null ? null
                 : (VariableExpr) nestClause.getPositionalVariable().accept(this, arg);
         Expression conditionExpresion = (Expression) nestClause.getConditionExpression().accept(this, arg);
-        return new NestClause(nestClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
+        NestClause copy = new NestClause(nestClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
                 conditionExpresion);
+        copy.setSourceLocation(nestClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -125,13 +133,18 @@
         VariableExpr rightVar = (VariableExpr) unnestClause.getRightVariable().accept(this, arg);
         VariableExpr rightPositionVar = unnestClause.getPositionalVariable() == null ? null
                 : (VariableExpr) unnestClause.getPositionalVariable().accept(this, arg);
-        return new UnnestClause(unnestClause.getJoinType(), rightExpression, rightVar, rightPositionVar);
+        UnnestClause copy = new UnnestClause(unnestClause.getJoinType(), rightExpression, rightVar, rightPositionVar);
+        copy.setSourceLocation(unnestClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public Projection visit(Projection projection, Void arg) throws CompilationException {
-        return new Projection(projection.star() ? null : (Expression) projection.getExpression().accept(this, arg),
-                projection.getName(), projection.star(), projection.varStar());
+        Projection copy =
+                new Projection(projection.star() ? null : (Expression) projection.getExpression().accept(this, arg),
+                        projection.getName(), projection.star(), projection.varStar());
+        copy.setSourceLocation(projection.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -170,8 +183,10 @@
             havingClause = (HavingClause) selectBlock.getHavingClause().accept(this, arg);
         }
         selectCluase = (SelectClause) selectBlock.getSelectClause().accept(this, arg);
-        return new SelectBlock(selectCluase, fromClause, letClauses, whereClause, gbyClause, gbyLetClauses,
+        SelectBlock copy = new SelectBlock(selectCluase, fromClause, letClauses, whereClause, gbyClause, gbyLetClauses,
                 havingClause);
+        copy.setSourceLocation(selectBlock.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -184,12 +199,16 @@
         if (selectClause.selectRegular()) {
             selectRegular = (SelectRegular) selectClause.getSelectRegular().accept(this, arg);
         }
-        return new SelectClause(selectElement, selectRegular, selectClause.distinct());
+        SelectClause copy = new SelectClause(selectElement, selectRegular, selectClause.distinct());
+        copy.setSourceLocation(selectClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public SelectElement visit(SelectElement selectElement, Void arg) throws CompilationException {
-        return new SelectElement((Expression) selectElement.getExpression().accept(this, arg));
+        SelectElement copy = new SelectElement((Expression) selectElement.getExpression().accept(this, arg));
+        copy.setSourceLocation(selectElement.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -198,7 +217,9 @@
         for (Projection projection : selectRegular.getProjections()) {
             projections.add((Projection) projection.accept(this, arg));
         }
-        return new SelectRegular(projections);
+        SelectRegular copy = new SelectRegular(projections);
+        copy.setSourceLocation(selectRegular.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -221,28 +242,39 @@
             }
             rightInputs.add(new SetOperationRight(right.getSetOpType(), right.isSetSemantics(), newRightInput));
         }
-        return new SelectSetOperation(newLeftInput, rightInputs);
+        SelectSetOperation copy = new SelectSetOperation(newLeftInput, rightInputs);
+        copy.setSourceLocation(selectSetOperation.getSourceLocation());
+        return copy;
     }
 
     @Override
     public HavingClause visit(HavingClause havingClause, Void arg) throws CompilationException {
-        return new HavingClause((Expression) havingClause.getFilterExpression().accept(this, arg));
+        HavingClause copy = new HavingClause((Expression) havingClause.getFilterExpression().accept(this, arg));
+        copy.setSourceLocation(havingClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public Query visit(Query q, Void arg) throws CompilationException {
-        return new Query(q.isExplain(), q.isTopLevel(), (Expression) q.getBody().accept(this, arg), q.getVarCounter(),
-                q.getExternalVars());
+        Query copy = new Query(q.isExplain(), q.isTopLevel(), (Expression) q.getBody().accept(this, arg),
+                q.getVarCounter(), q.getExternalVars());
+        copy.setSourceLocation(q.getSourceLocation());
+        return copy;
     }
 
     @Override
     public FunctionDecl visit(FunctionDecl fd, Void arg) throws CompilationException {
-        return new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        FunctionDecl copy =
+                new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        copy.setSourceLocation(fd.getSourceLocation());
+        return copy;
     }
 
     @Override
     public WhereClause visit(WhereClause whereClause, Void arg) throws CompilationException {
-        return new WhereClause((Expression) whereClause.getWhereExpr().accept(this, arg));
+        WhereClause copy = new WhereClause((Expression) whereClause.getWhereExpr().accept(this, arg));
+        copy.setSourceLocation(whereClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -251,7 +283,9 @@
         for (Expression orderExpr : oc.getOrderbyList()) {
             newOrderbyList.add((Expression) orderExpr.accept(this, arg));
         }
-        return new OrderbyClause(newOrderbyList, oc.getModifierList());
+        OrderbyClause copy = new OrderbyClause(newOrderbyList, oc.getModifierList());
+        copy.setSourceLocation(oc.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -281,21 +315,27 @@
         for (Pair<Expression, Identifier> field : gc.getGroupFieldList()) {
             groupFieldList.add(new Pair<>((Expression) field.first.accept(this, arg), field.second));
         }
-        return new GroupbyClause(gbyPairList, decorPairList, withVarMap, groupVarExpr, groupFieldList,
+        GroupbyClause copy = new GroupbyClause(gbyPairList, decorPairList, withVarMap, groupVarExpr, groupFieldList,
                 gc.hasHashGroupByHint(), gc.isGroupAll());
+        copy.setSourceLocation(gc.getSourceLocation());
+        return copy;
     }
 
     @Override
     public LimitClause visit(LimitClause limitClause, Void arg) throws CompilationException {
         Expression limitExpr = (Expression) limitClause.getLimitExpr().accept(this, arg);
         Expression offsetExpr = limitClause.hasOffset() ? (Expression) limitClause.getOffset().accept(this, arg) : null;
-        return new LimitClause(limitExpr, offsetExpr);
+        LimitClause copy = new LimitClause(limitExpr, offsetExpr);
+        copy.setSourceLocation(limitClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public LetClause visit(LetClause letClause, Void arg) throws CompilationException {
-        return new LetClause((VariableExpr) letClause.getVarExpr().accept(this, arg),
+        LetClause copy = new LetClause((VariableExpr) letClause.getVarExpr().accept(this, arg),
                 (Expression) letClause.getBindingExpr().accept(this, arg));
+        copy.setSourceLocation(letClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -317,11 +357,7 @@
 
         // visit order by
         if (selectExpression.hasOrderby()) {
-            List<Expression> orderExprs = new ArrayList<>();
-            for (Expression orderExpr : selectExpression.getOrderbyClause().getOrderbyList()) {
-                orderExprs.add((Expression) orderExpr.accept(this, arg));
-            }
-            orderby = new OrderbyClause(orderExprs, selectExpression.getOrderbyClause().getModifierList());
+            orderby = (OrderbyClause) selectExpression.getOrderbyClause().accept(this, arg);
         }
 
         // visit limit
@@ -338,7 +374,9 @@
 
     @Override
     public ListConstructor visit(ListConstructor lc, Void arg) throws CompilationException {
-        return new ListConstructor(lc.getType(), copyExprList(lc.getExprList(), arg));
+        ListConstructor copy = new ListConstructor(lc.getType(), copyExprList(lc.getExprList(), arg));
+        copy.setSourceLocation(lc.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -349,13 +387,17 @@
                     (Expression) binding.getRightExpr().accept(this, arg));
             bindings.add(fb);
         }
-        return new RecordConstructor(bindings);
+        RecordConstructor copy = new RecordConstructor(bindings);
+        copy.setSourceLocation(rc.getSourceLocation());
+        return copy;
     }
 
     @Override
     public OperatorExpr visit(OperatorExpr operatorExpr, Void arg) throws CompilationException {
-        return new OperatorExpr(copyExprList(operatorExpr.getExprList(), arg), operatorExpr.getExprBroadcastIdx(),
-                operatorExpr.getOpList(), operatorExpr.isCurrentop());
+        OperatorExpr copy = new OperatorExpr(copyExprList(operatorExpr.getExprList(), arg),
+                operatorExpr.getExprBroadcastIdx(), operatorExpr.getOpList(), operatorExpr.isCurrentop());
+        copy.setSourceLocation(operatorExpr.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -363,7 +405,9 @@
         Expression conditionExpr = (Expression) ifExpr.getCondExpr().accept(this, arg);
         Expression thenExpr = (Expression) ifExpr.getThenExpr().accept(this, arg);
         Expression elseExpr = (Expression) ifExpr.getElseExpr().accept(this, arg);
-        return new IfExpr(conditionExpr, thenExpr, elseExpr);
+        IfExpr copy = new IfExpr(conditionExpr, thenExpr, elseExpr);
+        copy.setSourceLocation(ifExpr.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -375,7 +419,9 @@
             quantifiedPairs.add(new QuantifiedPair(var, expr));
         }
         Expression condition = (Expression) qe.getSatisfiesExpr().accept(this, arg);
-        return new QuantifiedExpression(qe.getQuantifier(), quantifiedPairs, condition);
+        QuantifiedExpression copy = new QuantifiedExpression(qe.getQuantifier(), quantifiedPairs, condition);
+        copy.setSourceLocation(qe.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -384,24 +430,31 @@
         for (Expression expr : callExpr.getExprList()) {
             newExprList.add((Expression) expr.accept(this, arg));
         }
-        return new CallExpr(callExpr.getFunctionSignature(), newExprList);
+        CallExpr copy = new CallExpr(callExpr.getFunctionSignature(), newExprList);
+        copy.setSourceLocation(callExpr.getSourceLocation());
+        return copy;
     }
 
     @Override
     public VariableExpr visit(VariableExpr varExpr, Void arg) throws CompilationException {
         VariableExpr clonedVar = new VariableExpr(new VarIdentifier(varExpr.getVar()));
+        clonedVar.setSourceLocation(varExpr.getSourceLocation());
         clonedVar.setIsNewVar(varExpr.getIsNewVar());
         return clonedVar;
     }
 
     @Override
     public UnaryExpr visit(UnaryExpr u, Void arg) throws CompilationException {
-        return new UnaryExpr(u.getExprType(), (Expression) u.getExpr().accept(this, arg));
+        UnaryExpr copy = new UnaryExpr(u.getExprType(), (Expression) u.getExpr().accept(this, arg));
+        copy.setSourceLocation(u.getSourceLocation());
+        return copy;
     }
 
     @Override
     public FieldAccessor visit(FieldAccessor fa, Void arg) throws CompilationException {
-        return new FieldAccessor((Expression) fa.getExpr().accept(this, arg), fa.getIdent());
+        FieldAccessor copy = new FieldAccessor((Expression) fa.getExpr().accept(this, arg), fa.getIdent());
+        copy.setSourceLocation(fa.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -411,7 +464,9 @@
         if (ia.getIndexExpr() != null) {
             indexExpr = (Expression) ia.getIndexExpr().accept(this, arg);
         }
-        return new IndexAccessor(expr, indexExpr);
+        IndexAccessor copy = new IndexAccessor(expr, indexExpr);
+        copy.setSourceLocation(ia.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -420,7 +475,9 @@
         List<Expression> whenExprList = copyExprList(caseExpr.getWhenExprs(), arg);
         List<Expression> thenExprList = copyExprList(caseExpr.getThenExprs(), arg);
         Expression elseExpr = (Expression) caseExpr.getElseExpr().accept(this, arg);
-        return new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        CaseExpression copy = new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        copy.setSourceLocation(caseExpr.getSourceLocation());
+        return copy;
     }
 
     private List<Expression> copyExprList(List<Expression> exprs, Void arg) throws CompilationException {
@@ -430,5 +487,4 @@
         }
         return newExprList;
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index 5f6b75b..0973bec 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -250,7 +250,7 @@
     public Void visit(CallExpr pf, Integer step) throws CompilationException {
         FunctionSignature functionSignature = pf.getFunctionSignature();
         FunctionSignature normalizedFunctionSignature =
-                FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, false);
+                FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, false, pf.getSourceLocation());
         if (BuiltinFunctions.isBuiltinCompilerFunction(normalizedFunctionSignature, true)) {
             functionSignature = normalizedFunctionSignature;
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index 0222e0e..e00a3bd 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -77,7 +77,9 @@
             // therefore we propagate the substitution environment.
             currentEnv = p.second;
         }
-        return new Pair<>(new FromClause(newFromTerms), currentEnv);
+        FromClause newFromClause = new FromClause(newFromTerms);
+        newFromClause.setSourceLocation(fromClause.getSourceLocation());
+        return new Pair<>(newFromClause, currentEnv);
     }
 
     @Override
@@ -114,7 +116,9 @@
                 }
             }
         }
-        return new Pair<>(new FromTerm(newLeftExpr, newLeftVar, newLeftPosVar, newCorrelateClauses), currentEnv);
+        FromTerm newFromTerm = new FromTerm(newLeftExpr, newLeftVar, newLeftPosVar, newCorrelateClauses);
+        newFromTerm.setSourceLocation(fromTerm.getSourceLocation());
+        return new Pair<>(newFromTerm, currentEnv);
     }
 
     @Override
@@ -139,6 +143,7 @@
 
         JoinClause newJoinClause =
                 new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar, conditionExpr);
+        newJoinClause.setSourceLocation(joinClause.getSourceLocation());
         return new Pair<>(newJoinClause, currentEnv);
     }
 
@@ -162,9 +167,10 @@
         // The condition can refer to the newRightVar and newRightPosVar.
         Expression conditionExpr = (Expression) nestClause.getConditionExpression().accept(this, currentEnv).first;
 
-        NestClause newJoinClause =
+        NestClause newNestClause =
                 new NestClause(nestClause.getJoinType(), rightExpr, newRightVar, newRightPosVar, conditionExpr);
-        return new Pair<>(newJoinClause, currentEnv);
+        newNestClause.setSourceLocation(nestClause.getSourceLocation());
+        return new Pair<>(newNestClause, currentEnv);
     }
 
     @Override
@@ -185,9 +191,10 @@
             currentEnv.removeSubstitution(newRightPosVar);
         }
         // The condition can refer to the newRightVar and newRightPosVar.
-        UnnestClause newJoinClause =
+        UnnestClause newUnnestClause =
                 new UnnestClause(unnestClause.getJoinType(), rightExpr, newRightVar, newRightPosVar);
-        return new Pair<>(newJoinClause, currentEnv);
+        newUnnestClause.setSourceLocation(unnestClause.getSourceLocation());
+        return new Pair<>(newUnnestClause, currentEnv);
     }
 
     @Override
@@ -198,6 +205,7 @@
         }
         Projection newProjection = new Projection((Expression) projection.getExpression().accept(this, env).first,
                 projection.getName(), projection.star(), projection.varStar());
+        newProjection.setSourceLocation(projection.getSourceLocation());
         return new Pair<>(newProjection, env);
     }
 
@@ -255,8 +263,10 @@
         WhereClause whereClause = newWhere == null ? null : (WhereClause) newWhere.first;
         GroupbyClause groupbyClause = newGroupby == null ? null : (GroupbyClause) newGroupby.first;
         HavingClause havingClause = newHaving == null ? null : (HavingClause) newHaving.first;
-        return new Pair<>(new SelectBlock((SelectClause) newSelect.first, fromClause, newLetClauses, whereClause,
-                groupbyClause, newLetClausesAfterGby, havingClause), currentEnv);
+        SelectBlock newSelectBlock = new SelectBlock((SelectClause) newSelect.first, fromClause, newLetClauses,
+                whereClause, groupbyClause, newLetClausesAfterGby, havingClause);
+        newSelectBlock.setSourceLocation(selectBlock.getSourceLocation());
+        return new Pair<>(newSelectBlock, currentEnv);
     }
 
     @Override
@@ -266,13 +276,15 @@
         if (selectClause.selectElement()) {
             Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectElement =
                     selectClause.getSelectElement().accept(this, env);
-            return new Pair<>(new SelectClause((SelectElement) newSelectElement.first, null, distinct),
-                    newSelectElement.second);
+            SelectClause newSelectClause = new SelectClause((SelectElement) newSelectElement.first, null, distinct);
+            newSelectClause.setSourceLocation(selectClause.getSourceLocation());
+            return new Pair<>(newSelectClause, newSelectElement.second);
         } else {
             Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectRegular =
                     selectClause.getSelectRegular().accept(this, env);
-            return new Pair<>(new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct),
-                    newSelectRegular.second);
+            SelectClause newSelectClause = new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct);
+            newSelectClause.setSourceLocation(selectClause.getSourceLocation());
+            return new Pair<>(newSelectClause, newSelectRegular.second);
         }
     }
 
@@ -281,7 +293,9 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> newExpr =
                 selectElement.getExpression().accept(this, env);
-        return new Pair<>(new SelectElement((Expression) newExpr.first), newExpr.second);
+        SelectElement newSelectElement = new SelectElement((Expression) newExpr.first);
+        newSelectElement.setSourceLocation(selectElement.getSourceLocation());
+        return new Pair<>(newSelectElement, newExpr.second);
     }
 
     @Override
@@ -291,7 +305,9 @@
         for (Projection projection : selectRegular.getProjections()) {
             newProjections.add((Projection) projection.accept(this, env).first);
         }
-        return new Pair<>(new SelectRegular(newProjections), env);
+        SelectRegular newSelectRegular = new SelectRegular(newProjections);
+        newSelectRegular.setSourceLocation(selectRegular.getSourceLocation());
+        return new Pair<>(newSelectRegular, env);
     }
 
     @Override
@@ -329,6 +345,7 @@
             }
         }
         SelectSetOperation newSelectSetOperation = new SelectSetOperation(newLeftInput, newRightInputs);
+        newSelectSetOperation.setSourceLocation(selectSetOperation.getSourceLocation());
         return new Pair<>(newSelectSetOperation, selectSetOperation.hasRightInputs() ? env : leftResult.second);
     }
 
@@ -366,9 +383,10 @@
             newLimitClause = (LimitClause) p.first;
             currentEnv = p.second;
         }
-        return new Pair<>(
-                new SelectExpression(newLetList, newSelectSetOperation, newOrderbyClause, newLimitClause, subquery),
-                currentEnv);
+        SelectExpression newSelectExpression =
+                new SelectExpression(newLetList, newSelectSetOperation, newOrderbyClause, newLimitClause, subquery);
+        newSelectExpression.setSourceLocation(selectExpression.getSourceLocation());
+        return new Pair<>(newSelectExpression, currentEnv);
     }
 
     @Override
@@ -376,6 +394,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p = havingClause.getFilterExpression().accept(this, env);
         HavingClause newHavingClause = new HavingClause((Expression) p.first);
+        newHavingClause.setSourceLocation(havingClause.getSourceLocation());
         return new Pair<>(newHavingClause, p.second);
     }
 
@@ -389,7 +408,7 @@
                 VariableCloneAndSubstitutionUtil.visitAndCloneExprList(caseExpr.getThenExprs(), env, this);
         Expression elseExpr = (Expression) caseExpr.getElseExpr().accept(this, env).first;
         CaseExpression newCaseExpr = new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        newCaseExpr.setSourceLocation(caseExpr.getSourceLocation());
         return new Pair<>(newCaseExpr, env);
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
index 4542bab..bf57184 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
@@ -59,36 +59,45 @@
         LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
         arguments.add(argumentLiteral);
         CallExpr callExpression = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), arguments);
+        callExpression.setSourceLocation(deleteStmt.getSourceLocation());
 
         // From clause.
         VariableExpr var = deleteStmt.getVariableExpr();
         FromTerm fromTerm = new FromTerm(callExpression, var, null, null);
-        @SuppressWarnings("unchecked")
+        fromTerm.setSourceLocation(var.getSourceLocation());
         FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+        fromClause.setSourceLocation(var.getSourceLocation());
 
         // Where clause.
         WhereClause whereClause = null;
         Expression condition = deleteStmt.getCondition();
         if (condition != null) {
             whereClause = new WhereClause(condition);
+            whereClause.setSourceLocation(condition.getSourceLocation());
         }
 
         // Select clause.
         VariableExpr returnExpr = new VariableExpr(var.getVar());
         returnExpr.setIsNewVar(false);
+        returnExpr.setSourceLocation(var.getSourceLocation());
         SelectElement selectElement = new SelectElement(returnExpr);
+        selectElement.setSourceLocation(deleteStmt.getSourceLocation());
         SelectClause selectClause = new SelectClause(selectElement, null, false);
+        selectClause.setSourceLocation(deleteStmt.getSourceLocation());
 
         // Construct the select expression.
         SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, whereClause, null, null, null);
+        selectBlock.setSourceLocation(var.getSourceLocation());
         SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
+        selectSetOperation.setSourceLocation(var.getSourceLocation());
         SelectExpression selectExpression = new SelectExpression(null, selectSetOperation, null, null, false);
+        selectExpression.setSourceLocation(var.getSourceLocation());
         Query query = new Query(false, false, selectExpression, 0);
         query.setBody(selectExpression);
+        query.setSourceLocation(deleteStmt.getSourceLocation());
 
         // return the delete statement.
         deleteStmt.setQuery(query);
         return null;
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index bcb131b..81b21c8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -25,6 +25,7 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -54,6 +55,7 @@
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AbstractSqlppExpressionScopingVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
@@ -76,7 +78,7 @@
      */
     public AbstractSqlppExpressionScopingVisitor(LangRewritingContext context, List<VarIdentifier> externalVars) {
         this.context = context;
-        this.scopeChecker.setVarCounter(new Counter(context.getVarCounter()));
+        this.scopeChecker.setVarCounter(context.getVarCounter());
         if (externalVars != null) {
             for (VarIdentifier paramVar : externalVars) {
                 scopeChecker.getCurrentScope().addSymbolToScope(paramVar);
@@ -92,7 +94,7 @@
 
             // Merges the variables defined in the current from term into the scope of the current from clause.
             Scope scopeForFromTerm = scopeChecker.removeCurrentScope();
-            mergeScopes(scopeForFromClause, scopeForFromTerm);
+            mergeScopes(scopeForFromClause, scopeForFromTerm, fromTerm.getSourceLocation());
         }
         return null;
     }
@@ -105,12 +107,12 @@
 
         // Registers the data item variable.
         VariableExpr leftVar = fromTerm.getLeftVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), leftVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), leftVar.getVar(), leftVar.getSourceLocation());
 
         // Registers the positional variable
         if (fromTerm.hasPositionalVariable()) {
             VariableExpr posVar = fromTerm.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
         // Visits join/unnest/nest clauses.
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
@@ -130,16 +132,16 @@
 
         // Registers the data item variable.
         VariableExpr rightVar = joinClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (joinClause.hasPositionalVariable()) {
             // Registers the positional variable.
             VariableExpr posVar = joinClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
 
         Scope rightScope = scopeChecker.removeCurrentScope();
-        mergeScopes(leftScope, rightScope);
+        mergeScopes(leftScope, rightScope, joinClause.getRightExpression().getSourceLocation());
         scopeChecker.pushExistingScope(leftScope);
         // The condition expression can refer to the just registered variables
         // for the right branch.
@@ -156,12 +158,12 @@
 
         // Registers the data item variable.
         VariableExpr rightVar = nestClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (nestClause.hasPositionalVariable()) {
             // Registers the positional variable.
             VariableExpr posVar = nestClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
 
         // The condition expression can refer to the just registered variables
@@ -176,12 +178,12 @@
 
         // register the data item variable
         VariableExpr rightVar = unnestClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (unnestClause.hasPositionalVariable()) {
             // register the positional variable
             VariableExpr posVar = unnestClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
         return null;
     }
@@ -212,7 +214,6 @@
     public Expression visit(Query q, ILangExpression arg) throws CompilationException {
         q.setBody(visit(q.getBody(), q));
         q.setVarCounter(scopeChecker.getVarCounter());
-        context.setVarCounter(scopeChecker.getVarCounter());
         return null;
     }
 
@@ -234,7 +235,7 @@
             gbyKeyVarExpr.setExpr(visit(gbyKeyVarExpr.getExpr(), gc));
             VariableExpr gbyKeyVar = gbyKeyVarExpr.getVar();
             if (gbyKeyVar != null) {
-                addNewVarSymbolToScope(newScope, gbyKeyVar.getVar());
+                addNewVarSymbolToScope(newScope, gbyKeyVar.getVar(), gbyKeyVar.getSourceLocation());
             }
         }
         if (gc.hasGroupFieldList()) {
@@ -247,12 +248,13 @@
                 decorVarExpr.setExpr(visit(decorVarExpr.getExpr(), gc));
                 VariableExpr decorVar = decorVarExpr.getVar();
                 if (decorVar != null) {
-                    addNewVarSymbolToScope(newScope, decorVar.getVar());
+                    addNewVarSymbolToScope(newScope, decorVar.getVar(), decorVar.getSourceLocation());
                 }
             }
         }
         if (gc.hasGroupVar()) {
-            addNewVarSymbolToScope(newScope, gc.getGroupVar().getVar());
+            VariableExpr groupVar = gc.getGroupVar();
+            addNewVarSymbolToScope(newScope, groupVar.getVar(), groupVar.getSourceLocation());
         }
         if (gc.hasWithMap()) {
             Map<Expression, VariableExpr> newWithMap = new HashMap<>();
@@ -260,7 +262,7 @@
                 Expression expr = visit(entry.getKey(), arg);
                 Expression newKey = expr;
                 VariableExpr value = entry.getValue();
-                addNewVarSymbolToScope(newScope, value.getVar());
+                addNewVarSymbolToScope(newScope, value.getVar(), value.getSourceLocation());
                 newWithMap.put(newKey, value);
             }
             gc.setWithVarMap(newWithMap);
@@ -285,7 +287,8 @@
     public Expression visit(LetClause letClause, ILangExpression arg) throws CompilationException {
         scopeChecker.extendCurrentScope();
         letClause.setBindingExpr(visit(letClause.getBindingExpr(), letClause));
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), letClause.getVarExpr().getVar());
+        VariableExpr varExpr = letClause.getVarExpr();
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), varExpr.getVar(), varExpr.getSourceLocation());
         return null;
     }
 
@@ -327,7 +330,8 @@
         scopeChecker.createNewScope();
         for (QuantifiedPair pair : qe.getQuantifiedList()) {
             pair.setExpr(visit(pair.getExpr(), qe));
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), pair.getVarExpr().getVar());
+            VariableExpr varExpr = pair.getVarExpr();
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), varExpr.getVar(), varExpr.getSourceLocation());
         }
         qe.setSatisfiesExpr(visit(qe.getSatisfiesExpr(), qe));
         scopeChecker.removeCurrentScope();
@@ -338,7 +342,7 @@
     public Expression visit(VariableExpr varExpr, ILangExpression arg) throws CompilationException {
         String varName = varExpr.getVar().getValue();
         if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new CompilationException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, varExpr.getSourceLocation(),
                     "Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
         }
         Identifier ident = scopeChecker.lookupSymbol(varName);
@@ -361,7 +365,7 @@
         // Registers the (inserted) data item variable.
         VariableExpr bindingVar = insertStatement.getVar();
         if (bindingVar != null) {
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), bindingVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), bindingVar.getVar(), bindingVar.getSourceLocation());
         }
 
         // Visits the expression for the returning expression.
@@ -373,20 +377,22 @@
     }
 
     // Adds a new encountered alias identifier into a scope
-    private void addNewVarSymbolToScope(Scope scope, VarIdentifier var) throws CompilationException {
+    private void addNewVarSymbolToScope(Scope scope, VarIdentifier var, SourceLocation sourceLoc)
+            throws CompilationException {
         if (scope.findLocalSymbol(var.getValue()) != null) {
-            throw new CompilationException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     "Duplicate alias definitions: " + SqlppVariableUtil.toUserDefinedName(var.getValue()));
         }
         scope.addNewVarSymbolToScope(var);
     }
 
     // Merges <code>scopeToBeMerged</code> into <code>hostScope</code>.
-    private void mergeScopes(Scope hostScope, Scope scopeToBeMerged) throws CompilationException {
+    private void mergeScopes(Scope hostScope, Scope scopeToBeMerged, SourceLocation sourceLoc)
+            throws CompilationException {
         Set<String> symbolsToBeMerged = scopeToBeMerged.getLocalSymbols();
         for (String symbolToBeMerged : symbolsToBeMerged) {
             if (hostScope.findLocalSymbol(symbolToBeMerged) != null) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Duplicate alias definitions: " + SqlppVariableUtil.toUserDefinedName(symbolToBeMerged));
             }
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index b8b7622..6d9976a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -67,9 +67,12 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractLangExpression;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -175,6 +178,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 class SQLPPParser extends ScopeChecker implements IParser {
 
@@ -216,13 +220,14 @@
     };
 
     private static class FunctionName {
-       public String dataverse = null;
-       public String library = null;
-       public String function = null;
-       public String hint = null;
+       public String dataverse;
+       public String library;
+       public String function;
+       public String hint;
+       public SourceLocation sourceLoc;
     }
 
-    private static String getHint(Token t) {
+    private String getHint(Token t) {
         if (t.specialToken == null) {
             return null;
         }
@@ -234,7 +239,11 @@
         return s.substring(1).trim();
     }
 
-    private static IRecordFieldDataGen parseFieldDataGen(String hint) throws ParseException {
+    private Token getHintToken(Token t) {
+        return t.specialToken;
+    }
+
+    private IRecordFieldDataGen parseFieldDataGen(String hint, Token hintToken) throws ParseException {
       IRecordFieldDataGen rfdg = null;
       String splits[] = hint.split(" +");
       if (splits[0].equals(VAL_FILE_HINT)) {
@@ -260,7 +269,7 @@
         } else if (splits[1].equals("double")) {
           vt = FieldIntervalDataGen.ValueType.DOUBLE;
         } else {
-          throw new ParseException("Unknown type for interval data gen: " + splits[1]);
+          throw new SqlppParseException(getSourceLocation(hintToken), "Unknown type for interval data gen: " + splits[1]);
         }
         rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
       } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
@@ -294,10 +303,12 @@
         try {
             return Statement();
         } catch (Error e) {
-            // this is here as the JavaCharStream that's below the lexer somtimes throws Errors that are not handled
+            // this is here as the JavaCharStream that's below the lexer sometimes throws Errors that are not handled
             // by the ANTLR-generated lexer or parser (e.g it does this for invalid backslash u + 4 hex digits escapes)
             final String msg = e.getClass().getSimpleName() + (e.getMessage() != null ? ": " + e.getMessage() : "");
             throw new CompilationException(ErrorCode.PARSE_ERROR, msg);
+        } catch (SqlppParseException e) {
+            throw new CompilationException(ErrorCode.PARSE_ERROR, e.getSourceLocation(), getMessage(e));
         } catch (ParseException e) {
             throw new CompilationException(ErrorCode.PARSE_ERROR, getMessage(e));
         }
@@ -315,34 +326,43 @@
         int maxSize = appendExpected(expected, expectedTokenSequences, tokenImage);
         Token tok = currentToken.next;
         int line = tok.beginLine;
-        String message = "In line " + line + " >>" + getLine(line) + "<<" + sep + "Encountered ";
+        StringBuilder message = new StringBuilder(128);
+        message.append("In line ").append(line).append(" >>").append(getLine(line)).append("<<").append(sep).append("Encountered ");
         for (int i = 0; i < maxSize; i++) {
             if (i != 0) {
-                message += " ";
+                message.append(' ');
             }
             if (tok.kind == 0) {
-                message += fixQuotes(tokenImage[0]);
+                message.append(fixQuotes(tokenImage[0]));
                 break;
             }
             final String fixedTokenImage = tokenImage[tok.kind];
             if (! tok.image.equalsIgnoreCase(stripQuotes(fixedTokenImage))) {
-                message += fixQuotes(fixedTokenImage) + " ";
+                message.append(fixQuotes(fixedTokenImage)).append(' ');
             }
-            message += quot + addEscapes(tok.image) + quot;
+            message.append(quot).append(addEscapes(tok.image)).append(quot);
             tok = tok.next;
         }
-        message += " at column " + currentToken.next.beginColumn + "." + sep;
+        message.append(" at column ").append(currentToken.next.beginColumn).append('.').append(sep);
         if (REPORT_EXPECTED_TOKENS) {
             if (expectedTokenSequences.length == 1) {
-                message += "Was expecting:" + sep + "    ";
+                message.append("Was expecting:").append(sep).append("    ");
             } else {
-                message += "Was expecting one of:" + sep + "    ";
+                message.append("Was expecting one of:").append(sep).append("    ");
             }
-            message += expected.toString();
+            message.append(expected);
         }
-        return message;
+        return message.toString();
     }
 
+    protected static SourceLocation getSourceLocation(Token token) {
+        return token != null ? new SourceLocation(token.beginLine, token.beginColumn) : null;
+    }
+
+    protected static <T extends AbstractLangExpression> T addSourceLocation(T expr, Token token) {
+        expr.setSourceLocation(getSourceLocation(token));
+        return expr;
+    }
 }
 
 PARSER_END(SQLPPParser)
@@ -399,46 +419,51 @@
 
 DataverseDecl DataverseDeclaration() throws ParseException:
 {
+  Token startToken = null;
   String dvName = null;
 }
 {
-  <USE> dvName = Identifier()
+  <USE> { startToken = token; } dvName = Identifier()
     {
       defaultDataverse = dvName;
-      return new DataverseDecl(new Identifier(dvName));
+      DataverseDecl dvDecl = new DataverseDecl(new Identifier(dvName));
+      return addSourceLocation(dvDecl, startToken);
     }
 }
 
 Statement CreateStatement() throws ParseException:
 {
+  Token startToken = null;
   String hint = null;
-  boolean dgen = false;
+  Token hintToken = null;
+  boolean hintDGen = false;
   Statement stmt = null;
 }
 {
-  <CREATE>
+  <CREATE> { startToken = token; }
   (
     {
       hint = getHint(token);
-      if (hint != null && hint.startsWith(DGEN_HINT)) {
-        dgen = true;
+      if (hint != null) {
+        hintToken = getHintToken(token);
+        hintDGen = hint.startsWith(DGEN_HINT);
       }
     }
-    stmt = TypeSpecification(hint, dgen)
-    | stmt = NodegroupSpecification()
-    | stmt = DatasetSpecification()
-    | stmt = IndexSpecification()
-    | stmt = DataverseSpecification()
-    | stmt = FunctionSpecification()
-    | stmt = FeedSpecification()
-    | stmt = FeedPolicySpecification()
+    stmt = TypeSpecification(startToken, hint, hintDGen, hintToken)
+    | stmt = NodegroupSpecification(startToken)
+    | stmt = DatasetSpecification(startToken)
+    | stmt = IndexSpecification(startToken)
+    | stmt = DataverseSpecification(startToken)
+    | stmt = FunctionSpecification(startToken)
+    | stmt = FeedSpecification(startToken)
+    | stmt = FeedPolicySpecification(startToken)
   )
   {
     return stmt;
   }
 }
 
-TypeDecl TypeSpecification(String hint, boolean dgen) throws ParseException:
+TypeDecl TypeSpecification(Token startStmtToken, String hint, boolean dgen, Token hintToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
@@ -453,18 +478,18 @@
       if (dgen) {
         String splits[] = hint.split(" +");
         if (splits.length != 3) {
-          throw new ParseException("Expecting /*+ dgen <filename> <numberOfItems> */");
+          throw new SqlppParseException(getSourceLocation(hintToken), "Expecting /*+ dgen <filename> <numberOfItems> */");
         }
         filename = splits[1];
         numValues = Long.parseLong(splits[2]);
       }
       TypeDataGen tddg = new TypeDataGen(dgen, filename, numValues);
-      return new TypeDecl(nameComponents.first, nameComponents.second, typeExpr, tddg, ifNotExists);
+      TypeDecl stmt = new TypeDecl(nameComponents.first, nameComponents.second, typeExpr, tddg, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-
-NodegroupDecl NodegroupSpecification() throws ParseException:
+NodegroupDecl NodegroupSpecification(Token startStmtToken) throws ParseException:
 {
   String name = null;
   String tmp = null;
@@ -484,11 +509,12 @@
     }
   )*
     {
-      return new NodegroupDecl(new Identifier(name), ncNames, ifNotExists);
+      NodegroupDecl stmt = new NodegroupDecl(new Identifier(name), ncNames, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-DatasetDecl DatasetSpecification() throws ParseException:
+DatasetDecl DatasetSpecification(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
@@ -499,7 +525,7 @@
   Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
   String nodeGroupName = null;
   Map<String,String> hints = new HashMap<String,String>();
-  DatasetDecl dsetDecl = null;
+  DatasetDecl stmt = null;
   boolean autogenerated = false;
   Pair<Integer, List<String>> filterField = null;
   Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
@@ -519,7 +545,7 @@
         edd.setAdapter(adapterName);
         edd.setProperties(properties);
         try{
-        dsetDecl = new DatasetDecl(nameComponents.first,
+        stmt = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
@@ -532,7 +558,7 @@
                                    withRecord,
                                    ifNotExists);
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
         }
       }
 
@@ -544,8 +570,9 @@
         <WITH>
         name = Identifier()
         {
-            if(!name.toLowerCase().equals("meta")){
-                throw new ParseException("We can only support one additional associated field called \"meta\".");
+            if (!name.equalsIgnoreCase("meta")){
+                throw new SqlppParseException(getSourceLocation(startStmtToken),
+                    "We can only support one additional associated field called \"meta\".");
             }
         }
         <LEFTPAREN> metaTypeComponents = TypeName() <RIGHTPAREN>
@@ -559,14 +586,15 @@
     ( <WITH> withRecord = RecordConstructor() )?
       {
         if(filterField!=null && filterField.first!=0){
-          throw new ParseException("A filter field can only be a field in the main record of the dataset.");
+          throw new SqlppParseException(getSourceLocation(startStmtToken),
+            "A filter field can only be a field in the main record of the dataset.");
         }
         InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second,
                                                           primaryKeyFields.first,
                                                           autogenerated,
                                                           filterField == null? null : filterField.second);
         try{
-        dsetDecl = new DatasetDecl(nameComponents.first,
+        stmt = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
@@ -579,33 +607,34 @@
                                    withRecord,
                                    ifNotExists);
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
         }
       }
   )
     {
-      return dsetDecl;
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
 RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
 {
-  RefreshExternalDatasetStatement redss = new RefreshExternalDatasetStatement();
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   String datasetName = null;
 }
 {
-    <REFRESH> <EXTERNAL> Dataset() nameComponents = QualifiedName()
+    <REFRESH> { startToken = token; } <EXTERNAL> Dataset() nameComponents = QualifiedName()
     {
-    redss.setDataverseName(nameComponents.first);
-    redss.setDatasetName(nameComponents.second);
-    return redss;
+      RefreshExternalDatasetStatement stmt = new RefreshExternalDatasetStatement();
+      stmt.setDataverseName(nameComponents.first);
+      stmt.setDatasetName(nameComponents.second);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
-CreateIndexStatement IndexSpecification() throws ParseException:
+CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseException:
 {
-  CreateIndexStatement cis = new CreateIndexStatement();
+  CreateIndexStatement stmt = new CreateIndexStatement();
   String indexName = null;
   boolean ifNotExists = false;
   Pair<Identifier,Identifier> nameComponents = null;
@@ -621,13 +650,13 @@
     <ON> nameComponents = QualifiedName()
     <LEFTPAREN> ( fieldPair = OpenField()
       {
-        cis.addFieldExprPair(fieldPair.second);
-        cis.addFieldIndexIndicator(fieldPair.first);
+        stmt.addFieldExprPair(fieldPair.second);
+        stmt.addFieldIndexIndicator(fieldPair.first);
       }
     ) (<COMMA> fieldPair = OpenField()
       {
-        cis.addFieldExprPair(fieldPair.second);
-        cis.addFieldIndexIndicator(fieldPair.first);
+        stmt.addFieldExprPair(fieldPair.second);
+        stmt.addFieldIndexIndicator(fieldPair.first);
       }
     )* <RIGHTPAREN> ( <TYPE> indexType = IndexType() )? ( <ENFORCED> { enforced = true; } )?)
     |
@@ -642,16 +671,16 @@
     if (isPrimaryIdx && indexName == null) {
       indexName = "primary_idx_" + nameComponents.second;
     }
-    cis.setIndexName(new Identifier(indexName));
-    cis.setIfNotExists(ifNotExists);
-    cis.setDataverseName(nameComponents.first);
-    cis.setDatasetName(nameComponents.second);
+    stmt.setIndexName(new Identifier(indexName));
+    stmt.setIfNotExists(ifNotExists);
+    stmt.setDataverseName(nameComponents.first);
+    stmt.setDatasetName(nameComponents.second);
     if (indexType != null) {
-      cis.setIndexType(indexType.type);
-      cis.setGramLength(indexType.gramLength);
+      stmt.setIndexType(indexType.type);
+      stmt.setGramLength(indexType.gramLength);
     }
-    cis.setEnforced(enforced);
-    return cis;
+    stmt.setEnforced(enforced);
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
@@ -710,7 +739,7 @@
     }
 }
 
-CreateDataverseStatement DataverseSpecification() throws ParseException :
+CreateDataverseStatement DataverseSpecification(Token startStmtToken) throws ParseException :
 {
   String dvName = null;
   boolean ifNotExists = false;
@@ -719,11 +748,12 @@
   <DATAVERSE> dvName = Identifier()
   ifNotExists = IfNotExists()
     {
-      return new CreateDataverseStatement(new Identifier(dvName), null, ifNotExists);
+      CreateDataverseStatement stmt = new CreateDataverseStatement(new Identifier(dvName), null, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-CreateFunctionStatement FunctionSpecification() throws ParseException:
+CreateFunctionStatement FunctionSpecification(Token startStmtToken) throws ParseException:
 {
   FunctionSignature signature;
   boolean ifNotExists = false;
@@ -759,17 +789,18 @@
       getCurrentScope().addFunctionDescriptor(signature, false);
       removeCurrentScope();
       defaultDataverse = currentDataverse;
-      return new CreateFunctionStatement(signature, paramList, functionBody, functionBodyExpr, ifNotExists);
+      CreateFunctionStatement stmt = new CreateFunctionStatement(signature, paramList, functionBody, functionBodyExpr, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-CreateFeedStatement FeedSpecification() throws ParseException:
+CreateFeedStatement FeedSpecification(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   String adapterName = null;
   Map<String,String> properties = null;
-  CreateFeedStatement cfs = null;
+  CreateFeedStatement stmt = null;
   Pair<Identifier,Identifier> sourceNameComponents = null;
   RecordConstructor withRecord = null;
 }
@@ -778,15 +809,15 @@
   <WITH> withRecord = RecordConstructor()
   {
     try {
-      cfs = new CreateFeedStatement(nameComponents, withRecord, ifNotExists);
-      return cfs;
+      stmt = new CreateFeedStatement(nameComponents, withRecord, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     } catch (AlgebricksException e) {
-      throw new ParseException(e.getMessage());
+      throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
     }
   }
 }
 
-CreateFeedPolicyStatement FeedPolicySpecification() throws ParseException:
+CreateFeedPolicyStatement FeedPolicySpecification(Token startStmtToken) throws ParseException:
 {
   String policyName = null;
   String basePolicyName = null;
@@ -794,7 +825,7 @@
   String definition = null;
   boolean ifNotExists = false;
   Map<String,String> properties = null;
-  CreateFeedPolicyStatement cfps = null;
+  CreateFeedPolicyStatement stmt = null;
 }
 {
   (
@@ -802,22 +833,20 @@
       <FROM>
       (<POLICY> basePolicyName = Identifier() properties = Configuration() (<DEFINITION> definition = ConstantString())?
       {
-        cfps = new CreateFeedPolicyStatement(policyName,
+        stmt = new CreateFeedPolicyStatement(policyName,
                                    basePolicyName, properties, definition, ifNotExists);
       }
      | <PATH> sourcePolicyFile = ConstantString() (<DEFINITION> definition = ConstantString())?
        {
-        cfps = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
+        stmt = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
        }
      )
   )
     {
-      return cfps;
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-
-
 List<VarIdentifier> ParameterList() throws ParseException:
 {
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
@@ -896,7 +925,7 @@
     {
       arity = new Integer(token.image);
       if (arity < 0 && arity != FunctionIdentifier.VARARGS) {
-        throw new ParseException(" invalid arity:" + arity);
+        throw new SqlppParseException(getSourceLocation(token), "Invalid arity:" + arity);
       }
 
       // TODO use fctName.library
@@ -930,15 +959,16 @@
 
 Statement DropStatement() throws ParseException:
 {
+  Token startToken = null;
   String id = null;
   Pair<Identifier,Identifier> pairId = null;
   Triple<Identifier,Identifier,Identifier> tripleId = null;
   FunctionSignature funcSig = null;
   boolean ifExists = false;
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
-  <DROP>
+  <DROP> { startToken = token; }
   (
     Dataset() pairId = QualifiedName() ifExists = IfExists()
       {
@@ -974,7 +1004,7 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startToken);
   }
 }
 
@@ -994,54 +1024,59 @@
 
 InsertStatement InsertStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
 }
 {
-  <INSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+  <INSERT> { startToken = token; } <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
     query = Query(false)
     ( <RETURNING> returnExpression = Expression())?
     {
       if (returnExpression != null && var == null) {
-         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+        var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
       }
       query.setTopLevel(true);
-      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
-                                 returnExpression);
+      InsertStatement stmt = new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(),
+        var, returnExpression);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 UpsertStatement UpsertStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
 }
 {
-  <UPSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+  <UPSERT> { startToken = token; } <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
     query = Query(false)
     ( <RETURNING> returnExpression = Expression())?
     {
       if (returnExpression != null && var == null) {
-         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+        var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
       }
       query.setTopLevel(true);
-      return new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
-                                 returnExpression);
+      UpsertStatement stmt = new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(),
+        var, returnExpression);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 DeleteStatement DeleteStatement() throws ParseException:
 {
+  Token startToken = null;
   VariableExpr varExpr = null;
   Expression condition = null;
   Pair<Identifier, Identifier> nameComponents;
 }
 {
-  <DELETE>
+  <DELETE> { startToken = token; }
   <FROM> nameComponents  = QualifiedName()
          ((<AS>)? varExpr = Variable())?
   (<WHERE> condition = Expression())?
@@ -1050,14 +1085,17 @@
         varExpr = new VariableExpr();
         VarIdentifier var = SqlppVariableUtil.toInternalVariableIdentifier(nameComponents.second.getValue());
         varExpr.setVar(var);
+        addSourceLocation(varExpr, startToken);
       }
-      return new DeleteStatement(varExpr, nameComponents.first, nameComponents.second,
+      DeleteStatement stmt = new DeleteStatement(varExpr, nameComponents.first, nameComponents.second,
           condition, getVarCounter());
+      return addSourceLocation(stmt, startToken);
   }
 }
 
 UpdateStatement UpdateStatement() throws ParseException:
 {
+  Token startToken = null;
   VariableExpr vars;
   Expression target;
   Expression condition;
@@ -1065,7 +1103,7 @@
   List<UpdateClause> ucs = new ArrayList<UpdateClause>();
 }
 {
-  <UPDATE> vars = Variable() <IN> target = Expression()
+  <UPDATE> { startToken = token; } vars = Variable() <IN> target = Expression()
   <WHERE> condition = Expression()
   <LEFTPAREN> (uc = UpdateClause()
     {
@@ -1077,7 +1115,8 @@
     }
   )*) <RIGHTPAREN>
     {
-      return new UpdateStatement(vars, target, condition, ucs);
+      UpdateStatement stmt = new UpdateStatement(vars, target, condition, ucs);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
@@ -1108,18 +1147,21 @@
 
 Statement SetStatement() throws ParseException:
 {
+  Token startToken = null;
   String pn = null;
   String pv = null;
 }
 {
-  <SET> pn = Identifier() pv = ConstantString()
+  <SET> { startToken = token; } pn = Identifier() pv = ConstantString()
     {
-      return new SetStatement(pn, pv);
+      SetStatement stmt = new SetStatement(pn, pv);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 Statement WriteStatement() throws ParseException:
 {
+  Token startToken = null;
   String nodeName = null;
   String fileName = null;
   Query query;
@@ -1127,15 +1169,17 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-  <WRITE> <OUTPUT> <TO> nodeName = Identifier() <COLON> fileName = ConstantString()
+  <WRITE> { startToken = token; } <OUTPUT> <TO> nodeName = Identifier() <COLON> fileName = ConstantString()
     ( <USING> writerClass = ConstantString() )?
     {
-      return new WriteStatement(new Identifier(nodeName), fileName, writerClass);
+      WriteStatement stmt = new WriteStatement(new Identifier(nodeName), fileName, writerClass);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 LoadStatement LoadStatement() throws ParseException:
 {
+  Token startToken = null;
   Identifier dataverseName = null;
   Identifier datasetName = null;
   boolean alreadySorted = false;
@@ -1144,7 +1188,7 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-  <LOAD> Dataset() nameComponents = QualifiedName()
+  <LOAD> { startToken = token; } Dataset() nameComponents = QualifiedName()
     {
       dataverseName = nameComponents.first;
       datasetName = nameComponents.second;
@@ -1156,7 +1200,8 @@
     }
   )?
     {
-      return new LoadStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
+      LoadStatement stmt = new LoadStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
@@ -1174,77 +1219,68 @@
 
 Statement CompactStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
-  Statement stmt = null;
 }
 {
-  <COMPACT> Dataset() nameComponents = QualifiedName()
+  <COMPACT> { startToken = token; } Dataset() nameComponents = QualifiedName()
     {
-      stmt = new CompactStatement(nameComponents.first, nameComponents.second);
-    }
-    {
-      return stmt;
+      CompactStatement stmt = new CompactStatement(nameComponents.first, nameComponents.second);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 Statement ConnectionStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
-  Pair<Identifier,Identifier> datasetNameComponents = null;
-
-  Map<String,String> configuration = null;
+  Token startToken = null;
   Statement stmt = null;
-  String policy = null;
 }
 {
   (
-    <CONNECT> stmt = ConnectStatement()
-  | <DISCONNECT> stmt = DisconnectStatement()
-  | <START> stmt = StartStatement()
-  | <STOP> stmt = StopStatement()
+    <CONNECT> { startToken = token; } stmt = ConnectStatement(startToken)
+  | <DISCONNECT> { startToken = token; } stmt = DisconnectStatement(startToken)
+  | <START> { startToken = token; } stmt = StartStatement(startToken)
+  | <STOP> { startToken = token; } stmt = StopStatement(startToken)
   )
   {
     return stmt;
   }
 }
 
-Statement StartStatement() throws ParseException:
+Statement StartStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
 
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
   <FEED> feedNameComponents = QualifiedName()
   {
     stmt = new StartFeedStatement (feedNameComponents);
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-Statement StopStatement () throws ParseException:
+AbstractStatement StopStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
 
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
   <FEED> feedNameComponents = QualifiedName()
   {
     stmt = new StopFeedStatement (feedNameComponents);
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-
-Statement DisconnectStatement() throws ParseException:
+AbstractStatement DisconnectStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
-  Map<String,String> configuration = null;
-  Statement stmt = null;
-  String policy = null;
+  AbstractStatement stmt = null;
 }
 {
   (
@@ -1254,18 +1290,18 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-Statement ConnectStatement() throws ParseException:
+AbstractStatement ConnectStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
   List<FunctionSignature> appliedFunctions = new ArrayList<FunctionSignature>();
-  Statement stmt = null;
+  AbstractStatement stmt = null;
   String policy = null;
   String whereClauseBody = null;
   WhereClause whereClause = null;
@@ -1301,7 +1337,7 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
@@ -1368,9 +1404,9 @@
   ( value = ConstantString() | <INTEGER_LITERAL>
     {
       try {
-        value = "" + Long.valueOf(token.image);
+        value = String.valueOf(Long.parseLong(token.image));
       } catch (NumberFormatException nfe) {
-        throw new ParseException("inapproriate value: " + token.image);
+        throw new SqlppParseException(getSourceLocation(token), "inapproriate value: " + token.image);
       }
     }
   )
@@ -1414,6 +1450,7 @@
 
 RecordTypeDefinition RecordTypeDef() throws ParseException:
 {
+  Token startToken = null;
   RecordTypeDefinition recType = new RecordTypeDefinition();
   RecordTypeDefinition.RecordKind recordKind = null;
 }
@@ -1422,15 +1459,18 @@
     | <OPEN> { recordKind = RecordTypeDefinition.RecordKind.OPEN; } )?
    <LEFTBRACE>
     {
+      startToken = token;
       String hint = getHint(token);
       if (hint != null) {
         String splits[] = hint.split(" +");
         if (splits[0].equals(GEN_FIELDS_HINT)) {
           if (splits.length != 5) {
-            throw new ParseException("Expecting: /*+ gen-fields <type> <min> <max> <prefix>*/");
+            throw new SqlppParseException(getSourceLocation(getHintToken(token)),
+                "Expecting: /*+ gen-fields <type> <min> <max> <prefix>*/");
           }
           if (!splits[1].equals("int")) {
-            throw new ParseException("The only supported type for gen-fields is int.");
+            throw new SqlppParseException(getSourceLocation(getHintToken(token)),
+                "The only supported type for gen-fields is int.");
           }
           UndeclaredFieldsDataGen ufdg = new UndeclaredFieldsDataGen(UndeclaredFieldsDataGen.Type.INT,
              Integer.parseInt(splits[2]), Integer.parseInt(splits[3]), splits[4]);
@@ -1449,7 +1489,7 @@
         recordKind = RecordTypeDefinition.RecordKind.OPEN;
       }
       recType.setRecordKind(recordKind);
-      return recType;
+      return addSourceLocation(recType, startToken);
    }
 }
 
@@ -1463,7 +1503,7 @@
   fieldName = Identifier()
     {
       String hint = getHint(token);
-      IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint) : null;
+      IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint, token.specialToken) : null;
     }
   <COLON> type =  TypeExpr() (<QUES> { nullable = true; } )?
     {
@@ -1476,40 +1516,44 @@
   Pair<Identifier,Identifier> id = null;
 }
 {
- id = QualifiedName()
-   {
-     if (id.first == null && id.second.getValue().equalsIgnoreCase("int")) {
-        id.second.setValue("int64");
-     }
+  id = QualifiedName()
+  {
+    if (id.first == null && id.second.getValue().equalsIgnoreCase("int")) {
+      id.second.setValue("int64");
+    }
 
-     return new TypeReferenceExpression(id);
-   }
+    TypeReferenceExpression typeRef = new TypeReferenceExpression(id);
+    return addSourceLocation(typeRef, token);
+  }
 }
 
 OrderedListTypeDefinition OrderedListTypeDef() throws ParseException:
 {
+  Token startToken = null;
   TypeExpression type = null;
 }
 {
-  <LEFTBRACKET>
-    ( type =  TypeExpr() )
+  <LEFTBRACKET> { startToken = token; }
+    ( type = TypeExpr() )
   <RIGHTBRACKET>
   {
-    return new OrderedListTypeDefinition(type);
+    OrderedListTypeDefinition typeDef = new OrderedListTypeDefinition(type);
+    return addSourceLocation(typeDef, startToken);
   }
 }
 
-
 UnorderedListTypeDefinition UnorderedListTypeDef() throws ParseException:
 {
+  Token startToken = null;
   TypeExpression type = null;
 }
 {
-  <LEFTDBLBRACE>
+  <LEFTDBLBRACE> { startToken = token; }
     ( type =  TypeExpr() )
   <RIGHTDBLBRACE>
   {
-    return new UnorderedListTypeDefinition(type);
+    UnorderedListTypeDefinition typeDef = new UnorderedListTypeDefinition(type);
+    return addSourceLocation(typeDef, startToken);
   }
 }
 
@@ -1525,6 +1569,7 @@
   {
     FunctionName result = new FunctionName();
     result.hint = getHint(token);
+    result.sourceLoc = getSourceLocation(token);
   }
   ( <DOT> second = Identifier()
     {
@@ -1614,19 +1659,21 @@
 {
   List<String> exprList = new ArrayList<String>();
   String lit = null;
+  Token litToken = null;
   int source = 0;
 }
 {
   lit = Identifier()
   {
     boolean meetParens = false;
+    litToken = token;
   }
   (
     LOOKAHEAD(1)
     <LEFTPAREN><RIGHTPAREN>
     {
-        if(!lit.toLowerCase().equals("meta")){
-            throw new ParseException("The string before () has to be \"meta\".");
+        if(!lit.equalsIgnoreCase("meta")){
+            throw new SqlppParseException(getSourceLocation(litToken), "The string before () has to be \"meta\".");
         }
         meetParens = true;
         source = 1;
@@ -1659,7 +1706,6 @@
   }
 }
 
-
 String QuotedString() throws ParseException:
 {
 }
@@ -1670,7 +1716,6 @@
     }
 }
 
-
 String StringLiteral() throws ParseException:
 {
 }
@@ -1728,26 +1773,25 @@
 
 FunctionDecl FunctionDeclaration() throws ParseException:
 {
-  FunctionDecl funcDecl;
-  FunctionSignature signature;
+  Token startToken = null;
   String functionName;
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
   Expression funcBody;
   createNewScope();
 }
 {
-  <DECLARE> <FUNCTION>
+  <DECLARE> { startToken = token; } <FUNCTION>
     functionName = Identifier()
     paramList = ParameterList()
   <LEFTBRACE>
     (funcBody = SelectExpression(true) | funcBody = Expression())
   <RIGHTBRACE>
   {
-    signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
+    FunctionSignature signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
     getCurrentScope().addFunctionDescriptor(signature, false);
-    funcDecl = new FunctionDecl(signature, paramList, funcBody);
+    FunctionDecl stmt = new FunctionDecl(signature, paramList, funcBody);
     removeCurrentScope();
-    return funcDecl;
+    return addSourceLocation(stmt, startToken);
   }
 }
 
@@ -1775,12 +1819,12 @@
   )
   {
     query.setBody(expr);
+    query.setSourceLocation(expr.getSourceLocation());
     return query;
   }
 }
 
 
-
 Expression Expression():
 {
   Expression expr = null;
@@ -1790,7 +1834,6 @@
 (
     LOOKAHEAD(2)
     expr = OperatorExpr()
-    | expr = CaseExpr()
     | expr = QuantifiedExpression()
 )
     {
@@ -1798,9 +1841,7 @@
     }
 }
 
-
-
-Expression OperatorExpr()throws ParseException:
+Expression OperatorExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -1808,18 +1849,18 @@
 {
     operand = AndExpr()
     (
-
       <OR>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
         try{
             op.addOperator(token.image.toLowerCase());
         } catch (Exception e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
       }
 
@@ -1835,7 +1876,7 @@
     }
 }
 
-Expression AndExpr()throws ParseException:
+Expression AndExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -1843,18 +1884,18 @@
 {
     operand = NotExpr()
     (
-
       <AND>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
         try{
            op.addOperator(token.image.toLowerCase());
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
     }
 
@@ -1866,28 +1907,30 @@
     )*
 
     {
-      return op==null? operand: op;
+      return op==null ? operand: op;
     }
 }
 
-Expression NotExpr()throws ParseException:
+Expression NotExpr() throws ParseException:
 {
    Expression inputExpr;
    boolean not = false;
+   Token startToken = null;
 }
 {
-  (<NOT> { not = true; } )? inputExpr = RelExpr()
+  (<NOT> { not = true; startToken = token; } )? inputExpr = RelExpr()
   {
-    if(not){
+    if(not) {
         FunctionSignature signature = new FunctionSignature(BuiltinFunctions.NOT);
-        return new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(inputExpr)));
+        CallExpr callExpr = new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(inputExpr)));
+        return addSourceLocation(callExpr, startToken);
     } else {
         return inputExpr;
     }
   }
 }
 
-Expression RelExpr()throws ParseException:
+Expression RelExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -1923,11 +1966,12 @@
             op = new OperatorExpr();
             op.addOperand(operand, false); // broadcast is always for the right branch
             op.setCurrentop(true);
+            addSourceLocation(op, token);
           }
           try{
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -1945,8 +1989,7 @@
      }
 }
 
-
-Expression BetweenExpr()throws ParseException:
+Expression BetweenExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -1975,11 +2018,12 @@
             op = new OperatorExpr();
             op.addOperand(operand);
             op.setCurrentop(true);
+            addSourceLocation(op, token);
           }
           try{
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -2000,13 +2044,14 @@
        if (annotation != null) {
          op.addHint(annotation);
        }
-       return op==null? operand: op;
+       return op==null ? operand: op;
     }
 }
 
 Expression IsExpr() throws ParseException:
 {
-    Expression expr = null;
+    Token notToken = null;
+    CallExpr expr = null;
     Expression operand = null;
     boolean not = false;
     FunctionIdentifier fn = null;
@@ -2014,29 +2059,30 @@
 {
     operand = LikeExpr()
     ( <IS>
-        (<NOT> { not = true; })?
+        (<NOT> { not = true; notToken = token; })?
         (
             <NULL> { fn = BuiltinFunctions.IS_NULL; } |
             <MISSING> { fn = BuiltinFunctions.IS_MISSING; } |
             <UNKNOWN> { fn = BuiltinFunctions.IS_UNKNOWN; } |
-            <VALUED> { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
+            (<KNOWN> | <VALUED>) { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
         )
       {
         FunctionSignature signature = new FunctionSignature(fn);
         expr = new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(operand)));
-        if(not) {
+        addSourceLocation(expr, token);
+        if (not) {
            FunctionSignature notSignature = new FunctionSignature(BuiltinFunctions.NOT);
            expr = new CallExpr(notSignature, new ArrayList<Expression>(Collections.singletonList(expr)));
+           addSourceLocation(expr, notToken);
         }
       }
     )?
     {
-        return expr = expr==null? operand : expr;
+        return expr == null ? operand : expr;
     }
 }
 
-
-Expression LikeExpr()throws ParseException:
+Expression LikeExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -2051,15 +2097,16 @@
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
 
           String operator = token.image.toLowerCase();
           if (not) {
             operator = "not_" + operator;
           }
-          try{
+          try {
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -2074,7 +2121,7 @@
      }
 }
 
-Expression ConcatExpr()throws ParseException:
+Expression ConcatExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -2083,18 +2130,15 @@
     operand = AddExpr()
     (
       LOOKAHEAD(1)
-      (<CONCAT>)
+      <CONCAT>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            ((OperatorExpr)op).addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(OperatorType.CONCAT);
     }
     operand = AddExpr()
     {
@@ -2103,31 +2147,29 @@
     )*
 
     {
-       return op==null? operand: op;
+       return op == null ? operand : op;
     }
 }
 
-Expression AddExpr()throws ParseException:
+Expression AddExpr() throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = MultExpr()
     (
       LOOKAHEAD(1)
-      (<PLUS> | <MINUS>)
+      (<PLUS> { opType = OperatorType.PLUS; } | <MINUS> { opType = OperatorType.MINUS; } )
       {
         if (op == null) {
           op = new OperatorExpr();
-        op.addOperand(operand);
-        op.setCurrentop(true);
+          op.addOperand(operand);
+          op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            ((OperatorExpr)op).addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
 
     operand = MultExpr()
@@ -2137,30 +2179,33 @@
     )*
 
     {
-       return op==null? operand: op;
+       return op == null ? operand : op;
     }
 }
 
-Expression MultExpr()throws ParseException:
+Expression MultExpr() throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = ExponentExpr()
 
-    (( <MUL> | <DIV> | <MOD> | <IDIV>)
+    ( (
+        <MUL> { opType = OperatorType.MUL; } |
+        <DIVIDE> { opType = OperatorType.DIVIDE; } |
+        <DIV> { opType = OperatorType.DIV; } |
+        ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+      )
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            op.addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
     operand = ExponentExpr()
     {
@@ -2169,11 +2214,11 @@
     )*
 
      {
-       return op==null?operand:op;
+       return op == null ? operand : op;
      }
 }
 
-Expression ExponentExpr()throws ParseException:
+Expression ExponentExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -2186,12 +2231,9 @@
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            op.addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(OperatorType.CARET);
     }
     operand = UnaryExpr()
     {
@@ -2199,7 +2241,7 @@
     }
     )?
     {
-       return op==null?operand:op;
+       return op == null ? operand : op;
     }
 }
 
@@ -2213,99 +2255,96 @@
     ( (<PLUS> | <MINUS> | (<NOT> { not = true; } )? <EXISTS> )
     {
         String exprType = token.image.toLowerCase();
-        if(not){
+        if (not) {
            exprType = "not_" + exprType;
         }
         uexpr = new UnaryExpr();
-        try{
+        addSourceLocation(uexpr, token);
+        try {
             uexpr.setExprType(exprType);
         } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
     }
     )?
 
     expr = ValueExpr()
     {
-       if(uexpr==null){
-            return expr;
+       if (uexpr == null) {
+         return expr;
+       } else {
+         uexpr.setExpr(expr);
+         return uexpr;
        }
-       uexpr.setExpr(expr);
-       return uexpr;
     }
 }
 
-Expression ValueExpr()throws ParseException:
+Expression ValueExpr() throws ParseException:
 {
   Expression expr = null;
-  Identifier ident = null;
-  AbstractAccessor fa = null;
-  Expression indexExpr = null;
+  AbstractAccessor accessor = null;
 }
 {
   expr = PrimaryExpr() (
-     ident = Field()
-     {
-      fa = (fa == null ? new FieldAccessor(expr, ident)
-                       : new FieldAccessor(fa, ident));
-     }
-     | indexExpr = Index()
-     {
-      fa = (fa == null ? new IndexAccessor(expr, indexExpr)
-                       : new IndexAccessor(fa, indexExpr));
-     }
-    )*
-    {
-      return fa == null ? expr : fa;
-    }
+     accessor = FieldAccessor(accessor != null ? accessor : expr)
+     | accessor = IndexAccessor(accessor != null ? accessor : expr)
+  )*
+  {
+    return accessor == null ? expr : accessor;
+  }
 }
 
-Identifier Field() throws ParseException:
+FieldAccessor FieldAccessor(Expression inputExpr) throws ParseException:
 {
+  Token startToken = null;
   String ident = null;
 }
 {
-   <DOT> ident = Identifier()
+   <DOT> { startToken = token; } ident = Identifier()
     {
-      return new Identifier(ident);
+      FieldAccessor fa = new FieldAccessor(inputExpr, new Identifier(ident));
+      return addSourceLocation(fa, startToken);
     }
 }
 
-Expression Index() throws ParseException:
+IndexAccessor IndexAccessor(Expression inputExpr) throws ParseException:
 {
-    Expression expr = null;
+  Token startToken = null;
+  Expression expr = null;
 }
 {
-  <LEFTBRACKET> ( expr = Expression()
+  <LEFTBRACKET> { startToken = token; }
+  ( expr = Expression()
     {
-        if(expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
+        if (expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
         {
             Literal lit = ((LiteralExpr)expr).getValue();
-            if(lit.getLiteralType() != Literal.Type.INTEGER &&
-               lit.getLiteralType() != Literal.Type.LONG) {
-                throw new ParseException("Index should be an INTEGER");
+            if (lit.getLiteralType() != Literal.Type.INTEGER &&
+                lit.getLiteralType() != Literal.Type.LONG) {
+                throw new SqlppParseException(expr.getSourceLocation(), "Index should be an INTEGER");
             }
         }
     }
 
-      | <QUES> // ANY
+  | <QUES> // ANY
 
-      )
+  )
 
-   <RIGHTBRACKET>
-    {
-      return expr;
-    }
+  <RIGHTBRACKET>
+  {
+    IndexAccessor ia = new IndexAccessor(inputExpr, expr);
+    return addSourceLocation(ia, startToken);
+  }
 }
 
-
-Expression PrimaryExpr()throws ParseException:
+Expression PrimaryExpr() throws ParseException:
 {
   Expression expr = null;
 }
 {
   ( LOOKAHEAD(4)
     expr = FunctionCallExpr()
+  | expr = CaseExpr()
   | expr = Literal()
   | expr = VariableRef()
   | expr = ListConstructor()
@@ -2332,7 +2371,7 @@
         try {
             lit.setValue(new LongIntegerLiteral(Long.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <FLOAT_LITERAL>
@@ -2340,7 +2379,7 @@
         try {
             lit.setValue(new FloatLiteral(Float.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <DOUBLE_LITERAL>
@@ -2348,7 +2387,7 @@
         try {
             lit.setValue(new DoubleLiteral(Double.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <MISSING>
@@ -2369,54 +2408,53 @@
     }
   )
     {
-      return lit;
+      return addSourceLocation(lit, token);
     }
 }
 
-
 VariableExpr VariableRef() throws ParseException:
 {
-    VariableExpr varExp = new VariableExpr();
     VarIdentifier var = new VarIdentifier();
+    String id = null;
 }
 {
-    { String id = null; }
     (<IDENTIFIER> { id = token.image; } | id = QuotedString())
     {
      id = SqlppVariableUtil.toInternalVariableName(id); // Prefix user-defined variables with "$"
      Identifier ident = lookupSymbol(id);
      if (isInForbiddenScopes(id)) {
-       throw new ParseException("Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
+       throw new SqlppParseException(getSourceLocation(token),
+        "Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
      }
-     if(ident != null) { // exist such ident
+     VariableExpr varExp = new VariableExpr();
+     if (ident != null) { // exist such ident
        varExp.setVar((VarIdentifier)ident);
      } else {
        varExp.setVar(var);
        varExp.setIsNewVar(false);
        var.setValue(id);
      }
-     return varExp;
+     return addSourceLocation(varExp, token);
     }
 }
 
-
 VariableExpr Variable() throws ParseException:
 {
-    VariableExpr varExp = new VariableExpr();
     VarIdentifier var = new VarIdentifier();
+    String id = null;
 }
 {
-    { String id = null; }
     (<IDENTIFIER> { id = token.image; } | id = QuotedString())
     {
      id = SqlppVariableUtil.toInternalVariableName(id); // prefix user-defined variables with "$".
      Identifier ident = lookupSymbol(id);
+     VariableExpr varExp = new VariableExpr();
      if(ident != null) { // exist such ident
        varExp.setIsNewVar(false);
      }
      varExp.setVar(var);
      var.setValue(id);
-     return varExp;
+     return addSourceLocation(varExp, token);
     }
 }
 
@@ -2426,96 +2464,96 @@
 }
 {
     (
-        expr = OrderedListConstructor() | expr = UnorderedListConstructor()
+      expr = OrderedListConstructor() |
+      expr = UnorderedListConstructor()
     )
-
     {
       return expr;
     }
 }
 
-
 ListConstructor OrderedListConstructor() throws ParseException:
 {
-      ListConstructor expr = new ListConstructor();
-      List<Expression> exprList = null;
-      expr.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+  Token startToken = null;
+  List<Expression> exprList = null;
 }
 {
-    <LEFTBRACKET> exprList = ExpressionList() <RIGHTBRACKET>
-    {
-      expr.setExprList(exprList);
-      return expr;
-    }
+  <LEFTBRACKET> { startToken = token; }
+  exprList = ExpressionList()
+  <RIGHTBRACKET>
+  {
+    ListConstructor expr = new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, exprList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 ListConstructor UnorderedListConstructor() throws ParseException:
 {
-      ListConstructor expr = new ListConstructor();
-      List<Expression> exprList = null;
-      expr.setType(ListConstructor.Type.UNORDERED_LIST_CONSTRUCTOR);
+  Token startToken = null;
+  List<Expression> exprList = null;
 }
 {
-    <LEFTDBLBRACE> exprList = ExpressionList() <RIGHTDBLBRACE>
-    {
-      expr.setExprList(exprList);
-      return expr;
-    }
+  <LEFTDBLBRACE> { startToken = token; }
+  exprList = ExpressionList()
+  <RIGHTDBLBRACE>
+  {
+    ListConstructor expr = new ListConstructor(ListConstructor.Type.UNORDERED_LIST_CONSTRUCTOR, exprList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 List<Expression> ExpressionList() throws ParseException:
 {
-      Expression expr = null;
-      List<Expression> list = null;
-      List<Expression> exprList = new ArrayList<Expression>();
+  Expression expr = null;
+  List<Expression> exprList = new ArrayList<Expression>();
 }
 {
-    (
-      expr = Expression()
+  (
+    expr = Expression()
+    {
+      exprList.add(expr);
+    }
+    ( <COMMA> expr = Expression()
       {
         exprList.add(expr);
       }
-      ( <COMMA> expr = Expression()
-        {
-          exprList.add(expr);
-        }
-      )*
-    )?
-    {
-      return exprList;
-    }
+    )*
+  )?
+  {
+    return exprList;
+  }
 }
 
 RecordConstructor RecordConstructor() throws ParseException:
 {
-      RecordConstructor expr = new RecordConstructor();
-      FieldBinding tmp = null;
-      List<FieldBinding> fbList = new ArrayList<FieldBinding>();
+  Token startToken = null;
+  FieldBinding fb = null;
+  List<FieldBinding> fbList = new ArrayList<FieldBinding>();
 }
 {
-    <LEFTBRACE> (tmp = FieldBinding()
-    {
-      fbList.add(tmp);
-    }
-    (<COMMA> tmp = FieldBinding() { fbList.add(tmp);  })*)? <RIGHTBRACE>
-    {
-      expr.setFbList(fbList);
-      return expr;
-    }
+  <LEFTBRACE> { startToken = token; }
+  (
+    fb = FieldBinding() { fbList.add(fb); }
+    (<COMMA> fb = FieldBinding() { fbList.add(fb); })*
+  )?
+  <RIGHTBRACE>
+  {
+    RecordConstructor expr = new RecordConstructor(fbList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 FieldBinding FieldBinding() throws ParseException:
 {
-    Expression left, right;
+  Expression left, right;
 }
 {
-    left = Expression() <COLON> right = Expression()
-    {
-      return new FieldBinding(left, right);
-    }
+  left = Expression() <COLON> right = Expression()
+  {
+    return new FieldBinding(left, right);
+  }
 }
 
-
 Expression FunctionCallExpr() throws ParseException:
 {
   CallExpr callExpr;
@@ -2529,16 +2567,16 @@
 }
 {
   funcName = FunctionName()
-    {
-      hint = funcName.hint;
-    }
+  {
+    hint = funcName.hint;
+  }
   <LEFTPAREN> (
     ( <DISTINCT> { distinct = true; } )?
     ( tmp = Expression() | <MUL> { star = true; } )
     {
       if(star){
-        if(!funcName.function.toLowerCase().equals("count")){
-           throw new ParseException("The parameter * can only be used in COUNT().");
+        if(!funcName.function.equalsIgnoreCase("count")){
+           throw new SqlppParseException(getSourceLocation(token), "The parameter * can only be used in COUNT().");
         }
         argList.add(new LiteralExpr(new LongIntegerLiteral(1L)));
       } else {
@@ -2572,6 +2610,7 @@
           callExpr.addHint(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE);
         }
       }
+      callExpr.setSourceLocation(funcName.sourceLoc);
       return callExpr;
     }
 }
@@ -2592,19 +2631,19 @@
     }
 }
 
-
 Expression CaseExpr() throws ParseException:
 {
-   Expression conditionExpr = new LiteralExpr(TrueLiteral.INSTANCE);
+   Token startToken = null;
+   Expression conditionExpr = null;
    List<Expression> whenExprs = new ArrayList<Expression>();
    List<Expression> thenExprs = new ArrayList<Expression>();
    Expression elseExpr = null;
-
    Expression whenExpr = null;
    Expression thenExpr = null;
 }
 {
-   <CASE> (  conditionExpr = Expression() )?
+   <CASE> { startToken = token; }
+   ( conditionExpr = Expression() )?
    (
      <WHEN> whenExpr = Expression()
      {
@@ -2618,7 +2657,12 @@
    (<ELSE> elseExpr = Expression() )?
    <END>
    {
-     return new CaseExpression(conditionExpr, whenExprs, thenExprs, elseExpr);
+     if (conditionExpr == null) {
+        LiteralExpr litExpr = new LiteralExpr(TrueLiteral.INSTANCE);
+        conditionExpr = addSourceLocation(litExpr, startToken);
+     }
+     CaseExpression caseExpr = new CaseExpression(conditionExpr, whenExprs, thenExprs, elseExpr);
+     return addSourceLocation(caseExpr, startToken);
    }
 }
 
@@ -2636,7 +2680,10 @@
     (orderbyClause = OrderbyClause() {})?
     (limitClause = LimitClause() {})?
     {
-      return new SelectExpression(letClauses, selectSetOperation, orderbyClause, limitClause, subquery);
+      SelectExpression selectExpr =
+        new SelectExpression(letClauses, selectSetOperation, orderbyClause, limitClause, subquery);
+      selectExpr.setSourceLocation((!letClauses.isEmpty() ? letClauses.get(0) : selectSetOperation).getSourceLocation());
+      return selectExpr;
     }
 }
 
@@ -2669,7 +2716,9 @@
     }
   )*
   {
-    return new SelectSetOperation(setOperationInputLeft, setOperationRights);
+    SelectSetOperation selectSetOp = new SelectSetOperation(setOperationInputLeft, setOperationRights);
+    selectSetOp.setSourceLocation(selectBlockLeft.getSourceLocation());
+    return selectSetOp;
   }
 }
 
@@ -2693,10 +2742,11 @@
   GroupbyClause groupbyClause = null;
   List<LetClause> gbyLetClauses = null;
   HavingClause havingClause = null;
+  SourceLocation startSrcLoc = null;
 }
 {
   (
-     selectClause = SelectClause()
+     selectClause = SelectClause() { startSrcLoc = selectClause.getSourceLocation(); }
      (
         LOOKAHEAD(1)
         fromClause = FromClause()
@@ -2715,7 +2765,7 @@
         (havingClause = HavingClause())?
      )?
     |
-     fromClause = FromClause()
+     fromClause = FromClause() { startSrcLoc = fromClause.getSourceLocation(); }
      (
         LOOKAHEAD(1)
         fromLetClauses = LetClause()
@@ -2731,65 +2781,84 @@
      selectClause = SelectClause()
   )
   {
-    return new SelectBlock(selectClause, fromClause, fromLetClauses, whereClause, groupbyClause, gbyLetClauses, havingClause);
+    SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, fromLetClauses, whereClause, groupbyClause,
+      gbyLetClauses, havingClause);
+    selectBlock.setSourceLocation(startSrcLoc);
+    return selectBlock;
   }
 }
 
 SelectClause SelectClause() throws ParseException:
 {
+  Token startToken = null;
   SelectRegular selectRegular = null;
   SelectElement selectElement = null;
   boolean distinct = false;
 }
 {
-  <SELECT> (<ALL>|<DISTINCT> {distinct = true; } )?
+  <SELECT> { startToken = token; } (<ALL>|<DISTINCT> { distinct = true; } )?
   (
     selectRegular = SelectRegular()
     |
     selectElement = SelectElement()
   )?
   {
-    if(selectRegular == null && selectElement == null){
+    SourceLocation sourceLoc = getSourceLocation(startToken);
+    if (selectRegular == null && selectElement == null){
         Projection projection = new Projection(null, null, true, false);
+        projection.setSourceLocation(sourceLoc);
         List<Projection> projections = new ArrayList<Projection>();
         projections.add(projection);
         selectRegular = new SelectRegular(projections);
+        selectRegular.setSourceLocation(sourceLoc);
     }
-    return new SelectClause(selectElement, selectRegular, distinct);
+    SelectClause selectClause = new SelectClause(selectElement, selectRegular, distinct);
+    selectClause.setSourceLocation(sourceLoc);
+    return selectClause;
   }
 }
 
 SelectRegular SelectRegular() throws ParseException:
 {
+  SourceLocation startSrcLoc = null;
   List<Projection> projections = new ArrayList<Projection>();
+  Projection projection = null;
 }
 {
-   {
-      Projection projection = null;
-   }
-   projection = Projection() { projections.add(projection); }
-    ( LOOKAHEAD(2) <COMMA>
-       projection = Projection() {projections.add(projection);}
-    )*
+  projection = Projection()
   {
-    return new SelectRegular(projections);
+    projections.add(projection);
+    startSrcLoc = projection.getSourceLocation();
+  }
+  ( LOOKAHEAD(2) <COMMA> projection = Projection()
+    {
+      projections.add(projection);
+    }
+  )*
+  {
+    SelectRegular selectRegular = new SelectRegular(projections);
+    selectRegular.setSourceLocation(startSrcLoc);
+    return selectRegular;
   }
 }
 
 SelectElement SelectElement() throws ParseException:
 {
+  Token startToken = null;
   Expression expr = null;
   String name = null;
 }
 {
-  (<RAW>|<ELEMENT>|<VALUE>) expr = Expression()
+  (<RAW>|<ELEMENT>|<VALUE>) { startToken = token; } expr = Expression()
   {
-    return new SelectElement(expr);
+    SelectElement selectElement = new SelectElement(expr);
+    return addSourceLocation(selectElement, startToken);
   }
 }
 
 Projection Projection() throws ParseException :
 {
+  SourceLocation startSrcLoc = null;
   Expression expr = null;
   Identifier identifier = null;
   String name = null;
@@ -2798,8 +2867,8 @@
 }
 {
   (
-    <MUL> {star = true; }
-    | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> {varStar = true; }
+    <MUL> { star = true; startSrcLoc = getSourceLocation(token); }
+    | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { varStar = true; }
     | expr = Expression() ((<AS>)? name = Identifier())?
       {
         if (name == null) {
@@ -2811,12 +2880,15 @@
       }
   )
   {
-    return new Projection(expr, name, star, varStar);
+    Projection projection = new Projection(expr, name, star, varStar);
+    projection.setSourceLocation(expr != null ? expr.getSourceLocation() : startSrcLoc);
+    return projection;
   }
 }
 
 FromClause FromClause() throws ParseException :
 {
+  Token startToken = null;
   List<FromTerm> fromTerms = new ArrayList<FromTerm>();
   extendCurrentScope();
 }
@@ -2824,10 +2896,11 @@
   {
     FromTerm fromTerm = null;
   }
-    <FROM> fromTerm = FromTerm() { fromTerms.add(fromTerm); }
+    <FROM> { startToken = token; } fromTerm = FromTerm() { fromTerms.add(fromTerm); }
     (LOOKAHEAD(2) <COMMA> fromTerm = FromTerm() { fromTerms.add(fromTerm); } )*
   {
-    return new FromClause(fromTerms);
+    FromClause fromClause = new FromClause(fromTerms);
+    return addSourceLocation(fromClause, startToken);
   }
 }
 
@@ -2854,47 +2927,52 @@
      }
   )*
   {
-    if(leftVar==null){
+    if (leftVar == null) {
         leftVar = ExpressionToVariableUtil.getGeneratedVariable(leftExpr, true);
     }
-    return new FromTerm(leftExpr, leftVar, posVar, correlateClauses);
+    FromTerm fromTerm = new FromTerm(leftExpr, leftVar, posVar, correlateClauses);
+    fromTerm.setSourceLocation(leftExpr.getSourceLocation());
+    return fromTerm;
   }
 }
 
 JoinClause JoinClause(JoinType joinType) throws ParseException :
 {
+    Token startToken = null;
     Expression rightExpr = null;
     VariableExpr rightVar = null;
     VariableExpr posVar = null;
     Expression conditionExpr = null;
 }
 {
-  <JOIN> rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())? <ON> conditionExpr = Expression()
+  <JOIN> { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())? <ON> conditionExpr = Expression()
   {
     if(rightVar==null){
         rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
     }
-    return new JoinClause(joinType, rightExpr, rightVar, posVar, conditionExpr);
+    JoinClause joinClause = new JoinClause(joinType, rightExpr, rightVar, posVar, conditionExpr);
+    return addSourceLocation(joinClause, startToken);
   }
 }
 
 UnnestClause UnnestClause(JoinType joinType) throws ParseException :
 {
+    Token startToken = null;
     Expression rightExpr;
     VariableExpr rightVar;
     VariableExpr posVar = null;
 }
 {
-  (<UNNEST>|<CORRELATE>|<FLATTEN>) rightExpr = Expression() ((<AS>)? rightVar = Variable()) (<AT> posVar = Variable())?
+  (<UNNEST>|<CORRELATE>|<FLATTEN>) { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable()) (<AT> posVar = Variable())?
   {
-    if(rightVar==null){
-        rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
+    if (rightVar == null) {
+      rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
     }
-    return new UnnestClause(joinType, rightExpr, rightVar, posVar);
+    UnnestClause unnestClause = new UnnestClause(joinType, rightExpr, rightVar, posVar);
+    return addSourceLocation(unnestClause, startToken);
   }
 }
 
-
 JoinType JoinType() throws ParseException :
 {
    JoinType joinType = JoinType.INNER;
@@ -2922,30 +3000,32 @@
     }
 }
 
-WhereClause WhereClause()throws ParseException :
+WhereClause WhereClause() throws ParseException :
 {
-  WhereClause wc = new WhereClause();
+  Token startToken = null;
   Expression whereExpr;
 }
 {
-    <WHERE> whereExpr = Expression()
+    <WHERE> { startToken = token; } whereExpr = Expression()
     {
-      wc.setWhereExpr(whereExpr);
-      return wc;
+      WhereClause wc = new WhereClause(whereExpr);
+      return addSourceLocation(wc, startToken);
     }
 }
 
-OrderbyClause OrderbyClause()throws ParseException :
+OrderbyClause OrderbyClause() throws ParseException :
 {
+    Token startToken = null;
     OrderbyClause oc = new OrderbyClause();
     Expression orderbyExpr;
     List<Expression> orderbyList = new ArrayList<Expression>();
-    List<OrderbyClause.OrderModifier> modifierList = new ArrayList<OrderbyClause.OrderModifier >();
+    List<OrderbyClause.OrderModifier> modifierList = new ArrayList<OrderbyClause.OrderModifier>();
     int numOfOrderby = 0;
 }
 {
     <ORDER>
       {
+        startToken = token;
         String hint = getHint(token);
         if (hint != null) {
           if (hint.startsWith(INMEMORY_HINT)) {
@@ -2983,12 +3063,13 @@
     {
       oc.setModifierList(modifierList);
       oc.setOrderbyList(orderbyList);
-      return oc;
+      return addSourceLocation(oc, startToken);
     }
 }
 
 GroupbyClause GroupbyClause()throws ParseException :
 {
+    Token startToken = null;
     GroupbyClause gbc = new GroupbyClause();
     List<GbyVariableExpressionPair> vePairList = new ArrayList<GbyVariableExpressionPair>();
     VariableExpr var = null;
@@ -3006,6 +3087,7 @@
       }
     <GROUP>
       {
+         startToken = token;
          String hint = getHint(token);
          if (hint != null && hint.equals(HASH_GROUP_BY_HINT)) {
            gbc.setHashGroupByHint(true);
@@ -3067,39 +3149,43 @@
       gbc.setGroupVar(groupVar);
       gbc.setGroupFieldList(groupFieldList);
       replaceCurrentScope(newScope);
-      return gbc;
+      return addSourceLocation(gbc, startToken);
     }
 }
 
 HavingClause HavingClause() throws ParseException:
 {
+   Token startToken = null;
    Expression filterExpr = null;
 }
 {
-    <HAVING> filterExpr = Expression()
+    <HAVING> { startToken = token; } filterExpr = Expression()
     {
-       return new HavingClause(filterExpr);
+       HavingClause havingClause = new HavingClause(filterExpr);
+       return addSourceLocation(havingClause, startToken);
     }
 }
 
 LimitClause LimitClause() throws ParseException:
 {
+    Token startToken = null;
     LimitClause lc = new LimitClause();
     Expression expr;
     pushForbiddenScope(getCurrentScope());
 }
 {
-    <LIMIT> expr = Expression()    { lc.setLimitExpr(expr);    }
-    (<OFFSET> expr = Expression() { lc.setOffset(expr);    })?
+    <LIMIT> { startToken = token; } expr = Expression() { lc.setLimitExpr(expr); }
+    (<OFFSET> expr = Expression() { lc.setOffset(expr); })?
 
   {
     popForbiddenScope();
-    return lc;
+    return addSourceLocation(lc, startToken);
   }
 }
 
 QuantifiedExpression QuantifiedExpression()throws ParseException:
 {
+  Token startToken = null;
   QuantifiedExpression qc = new QuantifiedExpression();
   List<QuantifiedPair> quantifiedList = new ArrayList<QuantifiedPair>();
   Expression satisfiesExpr;
@@ -3112,8 +3198,8 @@
     createNewScope();
   }
 
-   (      ((<ANY>|<SOME>)  {  qc.setQuantifier(QuantifiedExpression.Quantifier.SOME);    })
-        | (<EVERY> {  qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY);    }))
+   ( ((<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
+     | (<EVERY> {  startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
     var = Variable() <IN> inExpr = Expression()
     {
       pair = new QuantifiedPair(var, inExpr);
@@ -3131,13 +3217,12 @@
        qc.setSatisfiesExpr(satisfiesExpr);
        qc.setQuantifiedList(quantifiedList);
        removeCurrentScope();
-       return qc;
+       return addSourceLocation(qc, startToken);
      }
 }
 
 LetClause LetElement() throws ParseException:
 {
-    LetClause lc = new LetClause();
     VariableExpr varExp;
     Expression beExp;
     extendCurrentScope();
@@ -3145,15 +3230,14 @@
 {
     varExp = Variable() <EQ> beExp = Expression()
     {
-      lc.setVarExpr(varExp);
-      lc.setBindingExpr(beExp);
+      LetClause lc = new LetClause(varExp, beExp);
+      lc.setSourceLocation(varExp.getSourceLocation());
       return lc;
     }
 }
 
 LetClause WithElement() throws ParseException:
 {
-    LetClause lc = new LetClause();
     VariableExpr varExp;
     Expression beExp;
     extendCurrentScope();
@@ -3161,8 +3245,8 @@
 {
     varExp = Variable() <AS> beExp = Expression()
     {
-      lc.setVarExpr(varExp);
-      lc.setBindingExpr(beExp);
+      LetClause lc = new LetClause(varExp, beExp);
+      lc.setSourceLocation(varExp.getSourceLocation());
       return lc;
     }
 }
@@ -3253,6 +3337,7 @@
   | <JOIN : "join">
   | <KEYWORD : "keyword">
   | <KEY : "key">
+  | <KNOWN : "known">
   | <LEFT : "left">
   | <LETTING : "letting">
   | <LET : "let">
@@ -3310,10 +3395,10 @@
 {
     <CARET : "^">
   | <CONCAT : "||">
-  | <DIV : "/">
-  | <IDIV : "idiv">
+  | <DIVIDE : "/">
+  | <DIV : "div">
   | <MINUS : "-">
-  | <MOD : "%">
+  | <MOD : "mod">
   | <MUL : "*">
   | <PLUS : "+">
 
@@ -3326,6 +3411,7 @@
   | <COLON : ":">
   | <COMMA : ",">
   | <DOT : ".">
+  | <PERCENT: "%">
   | <QUES : "?">
   | <SEMICOLON : ";">
   | <SHARP : "#">
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
index 16b09cb..5b680b2 100644
--- a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
@@ -78,8 +78,13 @@
     </dependency>
     <dependency>
       <groupId>org.ow2.asm</groupId>
-      <artifactId>asm-all</artifactId>
-      <version>5.1</version>
+      <artifactId>asm</artifactId>
+      <version>6.2</version>
+    </dependency>
+    <dependency>
+      <groupId>org.ow2.asm</groupId>
+      <artifactId>asm-tree</artifactId>
+      <version>6.2</version>
     </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorMissingCheckVisitor.java b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorMissingCheckVisitor.java
index 9339e8b..38b9751 100644
--- a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorMissingCheckVisitor.java
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/EvaluatorMissingCheckVisitor.java
@@ -109,7 +109,7 @@
             // Duplicate the top operand.
             mv.visitInsn(Opcodes.DUP);
             // Invoke the constructor of TypeChecker.
-            mv.visitMethodInsn(Opcodes.INVOKESPECIAL, TYPE_CHECKER_CLASS, CONSTRUCTOR, "()V", true);
+            mv.visitMethodInsn(Opcodes.INVOKESPECIAL, TYPE_CHECKER_CLASS, CONSTRUCTOR, "()V", false);
             // Putfield for the field typeChecker.
             mv.visitFieldInsn(Opcodes.PUTFIELD, className, TYPE_CHECKER_NAME, TYPE_CHECKER_DESC);
             // RETURN.
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
index 8ed9d50..f5d32ac 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
@@ -19,10 +19,10 @@
 package org.apache.asterix.lang.extension;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class EchoStatement implements Statement {
+public class EchoStatement extends AbstractStatement {
 
     private final String arg;
 
diff --git a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 3613166..044852b 100644
--- a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -124,6 +124,62 @@
         }
     }
 
+    public void getLastTokenImage(TokenImage image) {
+        if (bufpos >= tokenBegin) {
+            image.reset(buffer, tokenBegin, bufpos - tokenBegin);
+        } else {
+            image.reset(buffer, tokenBegin, bufsize - tokenBegin, buffer, 0, bufpos);
+        }
+    }
+
+    public static class TokenImage{
+        private char[] buffer;
+        private int begin;
+        private int length;
+
+        // used to hold circular tokens
+        private char[] tmpBuffer;
+
+        private static final double TMP_BUFFER_GROWTH = 1.5;
+
+        public void reset(char[] buffer, int begin, int length) {
+            this.buffer = buffer;
+            this.begin = begin;
+            this.length = length;
+        }
+
+        public void reset(char[] buffer, int begin, int length, char[] extraBuffer, int extraBegin, int extraLength) {
+            ensureTmpBufferSize(length + extraLength);
+            System.arraycopy(buffer, begin, tmpBuffer, 0, length);
+            System.arraycopy(extraBuffer, extraBegin, tmpBuffer, length, extraLength);
+            this.buffer = tmpBuffer;
+            this.begin = 0;
+            this.length = length + extraLength;
+        }
+
+        public char[] getBuffer() {
+            return buffer;
+        }
+
+        public int getBegin() {
+            return begin;
+        }
+
+        public int getLength() {
+            return length;
+        }
+
+
+        private void ensureTmpBufferSize(int size) {
+            int oldSize = tmpBuffer!=null?tmpBuffer.length:0;
+            if(oldSize < size) {
+                int newSize = Math.max((int)(oldSize * TMP_BUFFER_GROWTH), size);
+                tmpBuffer = new char[newSize];
+            }
+        }
+
+    }
+
     public int getColumn() {
         return column;
     }
@@ -262,4 +318,4 @@
         endOf_UNUSED_Buffer = bufsize;
         tokenBegin = 0;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
index be295d6..1cfb804 100644
--- a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
+++ b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
@@ -56,7 +56,7 @@
     public long allocate() {
         final LocalManager localManager = local.get();
         final @E@RecordManager recMgr = localManager.mgr;
-        final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7fff) : 0;
+        final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7ffe + 1) : 1;
         final int localId = recMgr.allocate();
 
         long result = TypeUtil.Global.build(localManager.arenaId, allocId, localId);
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 94c2668..da564d1 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -105,6 +105,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-core</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index a6a708f..4474684 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -97,7 +97,6 @@
     private final MetadataCache cache = new MetadataCache();
     protected final Collection<IAsterixStateProxy> proxies;
     protected IMetadataNode metadataNode;
-    private final ReadWriteLock metadataLatch;
     protected boolean rebindMetadataNode = false;
 
     // TODO(mblow): replace references of this (non-constant) field with a method,
@@ -117,7 +116,6 @@
             throw new IllegalArgumentException("Null / empty list of proxies given to MetadataManager");
         }
         this.proxies = proxies;
-        this.metadataLatch = new ReentrantReadWriteLock(true);
     }
 
     protected abstract TxnId createTxnId();
@@ -725,26 +723,6 @@
     }
 
     @Override
-    public void acquireWriteLatch() {
-        metadataLatch.writeLock().lock();
-    }
-
-    @Override
-    public void releaseWriteLatch() {
-        metadataLatch.writeLock().unlock();
-    }
-
-    @Override
-    public void acquireReadLatch() {
-        metadataLatch.readLock().lock();
-    }
-
-    @Override
-    public void releaseReadLatch() {
-        metadataLatch.readLock().unlock();
-    }
-
-    @Override
     public FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
             throws AlgebricksException {
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
index 7103fd8..baface2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
@@ -82,7 +82,7 @@
     void add(Dataset dataset) throws HyracksDataException;
 
     /**
-     * Remove dataset to the list of associated datasets
+     * Remove dataset from the list of associated datasets
      *
      * @param dataset
      *            the dataset to add
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 966792e..a646893 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -519,14 +519,6 @@
 
     int getMostRecentDatasetId() throws AlgebricksException;
 
-    void acquireWriteLatch();
-
-    void releaseWriteLatch();
-
-    void acquireReadLatch();
-
-    void releaseReadLatch();
-
     /**
      * Removes a library , acquiring local locks on behalf of the given transaction
      * id.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 980d36d..95526f4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -24,17 +24,19 @@
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 import org.apache.asterix.external.api.IAdapterFactory;
@@ -66,7 +68,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -78,7 +79,6 @@
 import org.apache.hyracks.storage.am.common.build.IndexBuilder;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -324,10 +324,10 @@
         ILSMOperationTrackerFactory opTrackerFactory =
                 index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(datasetId)
                         : new SecondaryIndexOperationTrackerFactory(datasetId);
-        ILSMComponentIdGeneratorFactory idGeneratorProvider =
-                new DatasetLSMComponentIdGeneratorFactory(index.getDatasetId().getId());
+        ILSMComponentIdGeneratorFactory idGeneratorProvider = new DatasetLSMComponentIdGeneratorFactory(datasetId);
+        DatasetInfoProvider datasetInfoProvider = new DatasetInfoProvider(datasetId);
         ILSMIOOperationCallbackFactory ioOpCallbackFactory =
-                new LSMBTreeIOOperationCallbackFactory(idGeneratorProvider);
+                new LSMIndexIOOperationCallbackFactory(idGeneratorProvider, datasetInfoProvider);
         IStorageComponentProvider storageComponentProvider = appContext.getStorageComponentProvider();
         if (isNewUniverse()) {
             final double bloomFilterFalsePositiveRate =
@@ -375,35 +375,23 @@
         // as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then
         // to INDEX_DATASET.
         MetadataTransactionContext mdTxnCtx = null;
-        MetadataManager.INSTANCE.acquireWriteLatch();
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Starting DDL recovery ...");
-        }
-
+        LOGGER.info("Starting DDL recovery ...");
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
             for (Dataverse dataverse : dataverses) {
                 recoverDataverse(mdTxnCtx, dataverse);
             }
-            // the commit wasn't there before. yet, everything was working
-            // correctly!!!!!!!!!!!
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("Completed DDL recovery.");
-            }
+            LOGGER.info("Completed DDL recovery.");
         } catch (Exception e) {
             try {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.log(Level.ERROR, "Failure during DDL recovery", e);
-                }
+                LOGGER.error("Failure during DDL recovery", e);
                 MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
             } catch (Exception e2) {
                 e.addSuppressed(e2);
             }
-            throw new MetadataException(e);
-        } finally {
-            MetadataManager.INSTANCE.releaseWriteLatch();
+            throw MetadataException.create(e);
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 66283d6..88449f9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public abstract class DataSource implements IDataSource<DataSourceId> {
 
@@ -159,7 +160,8 @@
     public abstract Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index ddab5bc..62cce05 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -21,6 +21,8 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataManager;
@@ -42,10 +44,11 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class DatasetDataSource extends DataSource {
 
-    private Dataset dataset;
+    private final Dataset dataset;
 
     public DatasetDataSource(DataSourceId id, Dataset dataset, IAType itemType, IAType metaItemType,
             byte datasourceType, IDatasetDetails datasetDetails, INodeDomain datasetDomain) throws AlgebricksException {
@@ -92,11 +95,16 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
         switch (dataset.getDatasetType()) {
             case EXTERNAL:
+                if (tupleFilterFactory != null || outputLimit >= 0) {
+                    throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                            "Tuple filter and limit are not supported by ExternalDataSource");
+                }
                 Dataset externalDataset = ((DatasetDataSource) dataSource).getDataset();
                 String itemTypeName = externalDataset.getItemTypeName();
                 IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
@@ -117,7 +125,8 @@
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
-                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, false);
+                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+                        false);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 5c3ed56..81be1f7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -22,7 +22,8 @@
 import java.util.List;
 
 import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
@@ -48,6 +49,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class FeedDataSource extends DataSource implements IMutationDataSource {
 
@@ -158,10 +160,15 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
         try {
+            if (tupleFilterFactory != null || outputLimit >= 0) {
+                throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        "Tuple filter and limit are not supported by FeedDataSource");
+            }
             ARecordType feedOutputType = (ARecordType) itemType;
             ISerializerDeserializer payloadSerde =
                     metadataProvider.getDataFormat().getSerdeProvider().getSerializerDeserializer(feedOutputType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index d2b9871..3b5cf2e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -25,6 +25,8 @@
 import java.util.Set;
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 import org.apache.asterix.metadata.api.IDatasourceFunction;
 import org.apache.asterix.om.types.IAType;
@@ -44,6 +46,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public abstract class FunctionDataSource extends DataSource {
 
@@ -68,9 +71,14 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
+        if (tupleFilterFactory != null || outputLimit >= 0) {
+            throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                    "tuple filter and limit are not supported by FunctionDataSource");
+        }
         GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
         adapterFactory.setOutputType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
         IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 8fdcbbc..3460a46 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -24,6 +24,8 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -44,6 +46,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class LoadableDataSource extends DataSource {
 
@@ -124,9 +127,14 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
+        if (tupleFilterFactory != null || outputLimit >= 0) {
+            throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                    "tuple filter and limit are not supported by LoadableDataSource");
+        }
         LoadableDataSource alds = (LoadableDataSource) dataSource;
         ARecordType itemType = (ARecordType) alds.getLoadedType();
         IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index a081fb4..7f8d31d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -26,7 +26,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -38,6 +37,7 @@
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -132,6 +132,7 @@
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -380,10 +381,12 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
             IDataSource<DataSourceId> dataSource, List<LogicalVariable> scanVariables,
             List<LogicalVariable> projectVariables, boolean projectPushed, List<LogicalVariable> minFilterVars,
-            List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
+            List<LogicalVariable> maxFilterVars, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec,
+            Object implConfig) throws AlgebricksException {
         return ((DataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables, projectVariables,
-                projectPushed, minFilterVars, maxFilterVars, opSchema, typeEnv, context, jobSpec, implConfig);
+                projectPushed, minFilterVars, maxFilterVars, tupleFilterFactory, outputLimit, opSchema, typeEnv,
+                context, jobSpec, implConfig);
     }
 
     protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
@@ -433,7 +436,8 @@
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
             boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
+            int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            boolean isIndexOnlyPlan) throws AlgebricksException {
         boolean isSecondary = true;
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), dataset.getDatasetName());
@@ -477,8 +481,8 @@
             btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                     context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                    maxFilterFieldIndexes, propagateFilter, proceedIndexOnlyPlan, failValueForIndexOnlyPlan,
-                    successValueForIndexOnlyPlan);
+                    maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit, proceedIndexOnlyPlan,
+                    failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -1541,7 +1545,7 @@
         }
     }
 
-    private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+    public AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
             throws AlgebricksException {
         // No filtering condition.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index a531add..14f76eb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -79,6 +79,7 @@
         _builtinTypeMap.put("null", BuiltinType.ANULL);
         _builtinTypeMap.put("uuid", BuiltinType.AUUID);
         _builtinTypeMap.put("shortwithouttypeinfo", BuiltinType.SHORTWITHOUTTYPEINFO);
+        _builtinTypeMap.put("geometry", BuiltinType.AGEOMETRY);
     }
 
     private BuiltinTypeMap() {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 7981309..bcd05ee 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -27,17 +27,17 @@
 
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.NoOpFrameOperationCallbackFactory;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.utils.JobUtils;
@@ -98,6 +98,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
@@ -105,7 +106,6 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -338,12 +338,14 @@
      *            a mutable progress state used for error handling during the drop operation
      * @param hcc
      *            a client connection to hyracks master for job execution
+     * @param sourceLoc
      * @throws Exception
      *             if an error occur during the drop process or if the dataset can't be dropped for any reason
      */
     public void drop(MetadataProvider metadataProvider, MutableObject<MetadataTransactionContext> mdTxnCtx,
             List<JobSpecification> jobsToExecute, MutableBoolean bActiveTxn, MutableObject<ProgressState> progress,
-            IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup) throws Exception {
+            IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup, SourceLocation sourceLoc)
+            throws Exception {
         Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
         if (getDatasetType() == DatasetType.INTERNAL) {
             // prepare job spec(s) that would disconnect any active feeds involving the dataset.
@@ -362,7 +364,8 @@
                     MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (indexes.get(j).isSecondaryIndex()) {
-                    jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this));
+                    jobsToExecute
+                            .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
                 }
             }
             jobsToExecute.add(DatasetUtil.dropDatasetJobSpec(this, metadataProvider));
@@ -399,7 +402,8 @@
                     MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
-                    jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this));
+                    jobsToExecute
+                            .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
                 } else {
                     jobsToExecute.add(DatasetUtil.buildDropFilesIndexJobSpec(metadataProvider, this));
                 }
@@ -502,24 +506,9 @@
      * @throws AlgebricksException
      *             if the factory could not be created for the index/dataset combination
      */
+    @SuppressWarnings("squid:S1172")
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        switch (index.getIndexType()) {
-            case BTREE:
-                return getDatasetType() == DatasetType.EXTERNAL
-                        && !index.getIndexName().equals(IndexingConstants.getFilesIndexName(getDatasetName()))
-                                ? new LSMBTreeWithBuddyIOOperationCallbackFactory(getComponentIdGeneratorFactory())
-                                : new LSMBTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case RTREE:
-                return new LSMRTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case LENGTH_PARTITIONED_NGRAM_INVIX:
-            case LENGTH_PARTITIONED_WORD_INVIX:
-            case SINGLE_PARTITION_NGRAM_INVIX:
-            case SINGLE_PARTITION_WORD_INVIX:
-                return new LSMInvertedIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
-                        index.getIndexType().toString());
-        }
+        return new LSMIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 
     /**
@@ -538,6 +527,10 @@
         return new DatasetLSMComponentIdGeneratorFactory(getDatasetId());
     }
 
+    public IDatasetInfoProvider getDatasetInfoProvider() {
+        return new DatasetInfoProvider(getDatasetId());
+    }
+
     /**
      * Get search callback factory for this dataset with the passed index and operation
      *
@@ -563,7 +556,6 @@
             boolean proceedIndexOnlyPlan) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
             /**
-            /*
              * Due to the read-committed isolation level,
              * we may acquire very short duration lock(i.e., instant lock) for readers.
              */
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index bf19b03..3159030 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -22,6 +22,7 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
@@ -153,8 +154,18 @@
             ARecordType recType) throws AlgebricksException {
         Pair<IAType, Boolean> keyPairType = null;
         IAType subType = recType;
+        boolean nullable = false;
         for (int i = 0; i < fieldName.size(); i++) {
-            subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+            if (subType instanceof AUnionType) {
+                nullable = nullable || ((AUnionType) subType).isUnknownableType();
+                subType = ((AUnionType) subType).getActualType();
+            }
+            if (subType instanceof ARecordType) {
+                subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+            } else {
+                throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+            }
+
             if (subType == null) {
                 keyPairType = Index.getNonNullableType(fieldType);
                 break;
@@ -163,13 +174,16 @@
         if (subType != null) {
             keyPairType = Index.getNonNullableKeyFieldType(fieldName, recType);
         }
+        keyPairType.second = keyPairType.second || nullable;
         return keyPairType;
     }
 
     public static Pair<IAType, Boolean> getNonNullableKeyFieldType(List<String> expr, ARecordType recType)
             throws AlgebricksException {
         IAType keyType = Index.keyFieldType(expr, recType);
-        return getNonNullableType(keyType);
+        Pair<IAType, Boolean> pair = getNonNullableType(keyType);
+        pair.second = pair.second || recType.isSubFieldNullable(expr);
+        return pair;
     }
 
     private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 8a0cf84..3ae0fec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -296,7 +296,6 @@
         }
 
         MetadataTransactionContext ctx = null;
-        MetadataManager.INSTANCE.acquireReadLatch();
         try {
             ctx = MetadataManager.INSTANCE.beginTransaction();
             Datatype t = MetadataManager.INSTANCE.getDatatype(ctx, dataverseName, datatypeName);
@@ -314,8 +313,6 @@
                 }
                 throw new MetadataException(ErrorCode.FEED_CREATE_FEED_DATATYPE_ERROR, e, datatypeName);
             }
-        } finally {
-            MetadataManager.INSTANCE.releaseReadLatch();
         }
         return outputType;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index a3c3842..418bc50 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -63,6 +63,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
@@ -241,6 +242,7 @@
      * @param dataset
      * @param files
      * @param indexerDesc
+     * @param sourceLoc
      * @return
      * @throws AlgebricksException
      * @throws HyracksDataException
@@ -248,23 +250,27 @@
      */
     private static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> getIndexingOperator(
             MetadataProvider metadataProvider, JobSpecification jobSpec, IAType itemType, Dataset dataset,
-            List<ExternalFile> files, RecordDescriptor indexerDesc) throws HyracksDataException, AlgebricksException {
+            List<ExternalFile> files, RecordDescriptor indexerDesc, SourceLocation sourceLoc)
+            throws HyracksDataException, AlgebricksException {
         ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
         Map<String, String> configuration = externalDatasetDetails.getProperties();
         IAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
                 metadataProvider.getApplicationContext().getServiceContext(), externalDatasetDetails.getAdapter(),
                 configuration, (ARecordType) itemType, files, true, null);
-        return new Pair<>(new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
-                adapterFactory.getPartitionConstraint());
+        ExternalScanOperatorDescriptor scanOp =
+                new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory);
+        scanOp.setSourceLocation(sourceLoc);
+        return new Pair<>(scanOp, adapterFactory.getPartitionConstraint());
     }
 
     public static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
             JobSpecification spec, MetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
-            RecordDescriptor indexerDesc, List<ExternalFile> files) throws HyracksDataException, AlgebricksException {
+            RecordDescriptor indexerDesc, List<ExternalFile> files, SourceLocation sourceLoc)
+            throws HyracksDataException, AlgebricksException {
         return getIndexingOperator(metadataProvider, spec, itemType, dataset,
                 files == null ? MetadataManager.INSTANCE
                         .getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset) : files,
-                indexerDesc);
+                indexerDesc, sourceLoc);
     }
 
     /**
@@ -420,8 +426,8 @@
     }
 
     public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider)
-            throws AlgebricksException {
+            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
         // Create files list
         ArrayList<ExternalFile> files = new ArrayList<>();
 
@@ -441,7 +447,7 @@
         for (ExternalFile file : appendedFiles) {
             files.add(file);
         }
-        return IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, files);
+        return IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, files, sourceLoc);
     }
 
     public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index eb14bae..c29279b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobSpecification;
 
@@ -103,40 +104,41 @@
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
-            Dataset dataset) throws AlgebricksException {
+            Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
-            Dataset dataset, Set<DropOption> options) throws AlgebricksException {
+            Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
 
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        return buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        return buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null, sourceLoc);
     }
 
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, List<ExternalFile> files) throws AlgebricksException {
+            MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
+            throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper;
         if (dataset.isCorrelated()) {
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
-                    index, metadataProvider, physicalOptimizationConfig);
+                    index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         } else {
             secondaryIndexHelper = SecondaryTreeIndexOperationsHelper.createIndexOperationsHelper(dataset, index,
-                    metadataProvider, physicalOptimizationConfig);
+                    metadataProvider, physicalOptimizationConfig, sourceLoc);
         }
         if (files != null) {
             secondaryIndexHelper.setExternalFiles(files);
@@ -145,9 +147,9 @@
     }
 
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index cf6338a..56e967e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -53,7 +53,6 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 
 public class SecondaryBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
 
@@ -108,9 +107,11 @@
                 secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, dataflowHelperFactory,
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             }
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+            metaOp.setSourceLocation(sourceLoc);
             spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
             root = metaOp;
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -170,9 +171,10 @@
 
             // bulk load op ----> sink op
             sourceOp = targetOp;
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
 
             spec.addRoot(targetOp);
@@ -236,7 +238,7 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn);
+                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -274,9 +276,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
-                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
+                    .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), itemType, filterFieldName,
+                            numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 7791cad..12e21f6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -102,9 +102,11 @@
                 createTreeIndexBulkLoadOp(spec, metadataProvider, taggedSecondaryRecDesc,
                         createFieldPermutationForBulkLoadOp(), getNumSecondaryKeys(), numPrimaryKeys, false);
 
-        AlgebricksMetaOperatorDescriptor metaOp =
-                new AlgebricksMetaOperatorDescriptor(spec, 1, 0, new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                        new RecordDescriptor[] { taggedSecondaryRecDesc });
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
+        AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { taggedSecondaryRecDesc });
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -155,7 +157,7 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn);
+                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -185,7 +187,7 @@
         if (numFilterFields > 0) {
             secondaryFieldAccessEvalFactories[numSecondaryKeys] =
                     metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index b91d65f..c6e4322 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -85,11 +85,12 @@
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
-                    indexType, RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
+                    sourceLoc, indexType,
+                    RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
         }
         if (numSecondaryKeys > 1) {
-            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, numSecondaryKeys,
-                    indexType, 1);
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, sourceLoc,
+                    numSecondaryKeys, indexType, 1);
         }
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
@@ -111,7 +112,7 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), recordColumn);
+                    index.getKeyFieldNames().get(0), recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -123,7 +124,7 @@
         if (numFilterFields > 0) {
             secondaryFieldAccessEvalFactories[numSecondaryKeys] =
                     metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
@@ -244,8 +245,11 @@
                 createTreeIndexBulkLoadOp(spec, metadataProvider, taggedTokenKeyPairRecDesc,
                         createFieldPermutationForBulkLoadOp(), getNumSecondaryKeys(), numPrimaryKeys, true);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index bf5178c..7a855d5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -77,7 +77,8 @@
         int numSecondaryKeys = secondaryKeyFields.size();
         boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
         if (numSecondaryKeys != 1) {
-            throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, numSecondaryKeys);
+            throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, sourceLoc,
+                    numSecondaryKeys);
         }
         Pair<IAType, Boolean> spatialTypePair =
                 Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
@@ -89,7 +90,7 @@
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
         secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
                 metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR, sourceLoc);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
@@ -212,8 +213,11 @@
                 metadataProvider, secondaryRecDescConsideringPointMBR, createFieldPermutationForBulkLoadOp(),
                 numNestedSecondaryKeFieldsConsideringPointMBR, numPrimaryKeys, false);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 48982c0..f863589 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -47,6 +47,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.primitive.BooleanPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -177,6 +178,7 @@
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
         sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { getTaggedRecordDescriptor(enforcedRecDesc) });
     }
@@ -219,8 +221,10 @@
             sefs[i] = secondaryFieldAccessEvalFactories[i];
         }
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        assign.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixAssignOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
                 primaryPartitionConstraint);
         return asterixAssignOp;
@@ -231,6 +235,7 @@
         IOperatorDescriptor op = new LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor(spec,
                 taggedSecondaryRecDesc, MissingWriterFactory.INSTANCE, NUM_TAG_FIELDS, numSecondaryKeyFields,
                 numPrimaryKeyFields, hasBuddyBTree);
+        op.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, primaryPartitionConstraint);
         return op;
     }
@@ -248,6 +253,7 @@
         ExternalSortOperatorDescriptor sortOp =
                 new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
                         taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
@@ -265,6 +271,7 @@
                 new LSMSecondaryIndexBulkLoadOperatorDescriptor(spec, taggedSecondaryRecDesc, primaryIndexHelperFactory,
                         secondaryIndexHelperFactory, fieldPermutation, NUM_TAG_FIELDS, numSecondaryKeys, numPrimaryKeys,
                         hasBuddyBtree);
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -280,15 +287,17 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
         LSMBTreeDiskComponentScanOperatorDescriptor primaryScanOp = new LSMBTreeDiskComponentScanOperatorDescriptor(
                 spec, outRecDesc, indexHelperFactory, searchCallbackFactory);
+        primaryScanOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primaryScanOp,
                 primaryPartitionConstraint);
         return primaryScanOp;
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AlgebricksException {
+            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+            throws AlgebricksException {
 
-        SecondaryIndexOperationsHelper indexOperationsHelper = null;
+        SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
                 indexOperationsHelper =
@@ -306,8 +315,10 @@
                         physOptConf, metadataProvider);
                 break;
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+                        index.getIndexType());
         }
+        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 065eb72..5f9e6ef 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -67,6 +67,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -118,6 +119,7 @@
     protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
     protected int numPrimaryKeys;
+    protected SourceLocation sourceLoc;
 
     // Prevent public construction. Should be created via createIndexCreator().
     protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
@@ -142,7 +144,8 @@
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AlgebricksException {
+            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+            throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
@@ -162,8 +165,10 @@
                         new SecondaryInvertedIndexOperationsHelper(dataset, index, physOptConf, metadataProvider);
                 break;
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+                        index.getIndexType());
         }
+        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
@@ -176,6 +181,10 @@
 
     public abstract JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException;
 
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
     protected void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
         metaSerde =
@@ -288,8 +297,10 @@
             sefs[i] = secondaryFieldAccessEvalFactories[i];
         }
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        assign.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixAssignOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
                 primaryPartitionConstraint);
         return asterixAssignOp;
@@ -319,6 +330,7 @@
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
         sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { enforcedRecDesc });
     }
@@ -326,6 +338,7 @@
     protected IFunctionDescriptor createCastFunction(boolean strictCast) throws AlgebricksException {
         IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
                 .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
+        castFuncDesc.setSourceLocation(sourceLoc);
         castFuncDesc.setImmutableStates(enforcedItemType, itemType);
         return castFuncDesc;
     }
@@ -338,6 +351,7 @@
         }
         ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
                 physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+        sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
@@ -351,6 +365,7 @@
         LSMIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new LSMIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 primaryIndexDataflowHelperFactory, BulkLoadUsage.CREATE_INDEX, dataset.getDatasetId());
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -362,6 +377,7 @@
         ExternalIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider));
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -371,7 +387,9 @@
             RecordDescriptor secondaryRecDesc) throws AlgebricksException {
         IScalarEvaluatorFactory[] andArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
         NotDescriptor notDesc = new NotDescriptor();
+        notDesc.setSourceLocation(sourceLoc);
         IsUnknownDescriptor isUnknownDesc = new IsUnknownDescriptor();
+        isUnknownDesc.setSourceLocation(sourceLoc);
         for (int i = 0; i < numSecondaryKeyFields; i++) {
             // Access column i, and apply 'is not null'.
             ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
@@ -386,14 +404,17 @@
             // Create conjunctive condition where all secondary index keys must
             // satisfy 'is not null'.
             AndDescriptor andDesc = new AndDescriptor();
+            andDesc.setSourceLocation(sourceLoc);
             selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
         } else {
             selectCond = andArgsEvalFactories[0];
         }
         StreamSelectRuntimeFactory select =
                 new StreamSelectRuntimeFactory(selectCond, null, BinaryBooleanInspector.FACTORY, false, -1, null);
+        select.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixSelectOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
                 primaryPartitionConstraint);
         return asterixSelectOp;
@@ -420,7 +441,7 @@
         Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
         try {
             indexingOpAndConstraints = ExternalIndexingOperations.createExternalIndexingOp(spec, metadataProvider,
-                    dataset, itemType, indexerDesc, externalFiles);
+                    dataset, itemType, indexerDesc, externalFiles, sourceLoc);
         } catch (Exception e) {
             throw new AlgebricksException(e);
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index d31ca3b..878aab6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -88,11 +88,12 @@
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
-                    indexType, RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
+                    sourceLoc, indexType,
+                    RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
         }
         if (numSecondaryKeys > 1) {
-            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, numSecondaryKeys,
-                    indexType, 1);
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, sourceLoc,
+                    numSecondaryKeys, indexType, 1);
         }
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
@@ -113,7 +114,7 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), numPrimaryKeys);
+                    index.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -123,9 +124,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
-                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
+                    .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), itemType, filterFieldName,
+                            numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
@@ -242,8 +243,11 @@
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
                 secondaryPartitionConstraint);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -271,6 +275,7 @@
         BinaryTokenizerOperatorDescriptor tokenizerOp =
                 new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
                         primaryKeyFields, isPartitioned, false, false, MissingWriterFactory.INSTANCE);
+        tokenizerOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
         return tokenizerOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 8e6e0e9..1d6677e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -99,7 +99,7 @@
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
                 metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR, sourceLoc);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
@@ -228,8 +228,11 @@
             // Create secondary RTree bulk load op.
             TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation,
                     indexDataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+            metaOp.setSourceLocation(sourceLoc);
             // Connect the operators.
             spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -285,9 +288,11 @@
                 secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, indexDataflowHelperFactory,
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             }
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+            metaOp.setSourceLocation(sourceLoc);
             spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
             root = metaOp;
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index 185cf8c..d78e80c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -62,6 +62,7 @@
                 secondaryFileSplitProvider, resourceFactory, true);
         IndexCreateOperatorDescriptor secondaryIndexCreateOp =
                 new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
+        secondaryIndexCreateOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -78,6 +79,7 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
         // The index drop operation should be persistent regardless of temp datasets or permanent dataset.
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, options);
+        btreeDrop.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
                 splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
@@ -93,6 +95,7 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
         LSMTreeIndexCompactOperatorDescriptor compactOp =
                 new LSMTreeIndexCompactOperatorDescriptor(spec, dataflowHelperFactory);
+        compactOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
         spec.addRoot(compactOp);
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 45cf7c8..3be3e2f 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -122,5 +122,10 @@
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+      <version>2.0.0</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 95aef79..40b42eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -69,7 +69,7 @@
     private int[] openFieldNameLengths;
 
     private int numberOfOpenFields;
-    private RuntimeRecordTypeInfo recTypeInfo;
+    private final RuntimeRecordTypeInfo recTypeInfo;
 
     public RecordBuilder() {
         this.closedPartOutputStream = new ByteArrayAccessibleOutputStream();
@@ -302,12 +302,7 @@
 
     @Override
     public int getFieldId(String fieldName) {
-        for (int i = 0; i < recType.getFieldNames().length; i++) {
-            if (recType.getFieldNames()[i].equals(fieldName)) {
-                return i;
-            }
-        }
-        return -1;
+        return recType.getFieldIndex(fieldName);
     }
 
     public IBinaryHashFunction getFieldNameHashFunction() {
@@ -317,4 +312,4 @@
     public IBinaryComparator getFieldNameComparator() {
         return utf8Comparator;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
index e4cf000..1dcf6a6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.dataflow.data.common;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -55,8 +57,10 @@
                 try {
                     return env.getVarType(((VariableReferenceExpression) expr).getVariableReference());
                 } catch (Exception e) {
-                    throw new AlgebricksException("Could not resolve type for " + expr.toString() + ","
-                            + "please check whether the used variable has been defined!", e);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                            "Could not resolve type for " + expr.toString() + ","
+                                    + "please check whether the used variable has been defined!",
+                            e);
                 }
             default:
                 throw new IllegalStateException();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
index dfb1251..33bef5d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
@@ -42,6 +42,7 @@
         AggregateFunctionCallExpression agg = (AggregateFunctionCallExpression) expr;
         FunctionIdentifier fid = agg.getFunctionIdentifier();
         VariableReferenceExpression tempVarExpr = new VariableReferenceExpression(originalProducedVar);
+        tempVarExpr.setSourceLocation(agg.getSourceLocation());
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
         Mutable<ILogicalExpression> mutableExpression = new MutableObject<ILogicalExpression>(tempVarExpr);
         arguments.add(mutableExpression);
@@ -56,6 +57,9 @@
              */
             return null;
         }
-        return BuiltinFunctions.makeAggregateFunctionExpression(mergeFid, arguments);
+        AggregateFunctionCallExpression mergeExpr =
+                BuiltinFunctions.makeAggregateFunctionExpression(mergeFid, arguments);
+        mergeExpr.setSourceLocation(agg.getSourceLocation());
+        return mergeExpr;
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
index 21880dd..e58f210 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.PrintStream;
 
+import org.apache.asterix.dataflow.data.nontagged.printers.json.clean.AGeometryPrinterFactory;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -116,6 +117,9 @@
             case SHORTWITHOUTTYPEINFO:
                 ShortWithoutTypeInfoPrinterFactory.PRINTER.print(b, s, l, ps);
                 return true;
+            case GEOMETRY:
+                AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
             default:
                 return false;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
new file mode 100644
index 0000000..8e51e35
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.PrintStream;
+
+public class AGeometryPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final AGeometryPrinterFactory INSTANCE = new AGeometryPrinterFactory();
+
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ByteArrayInputStream inStream = new ByteArrayInputStream(b, s + 1, l - 1);
+        DataInput dataIn = new DataInputStream(inStream);
+        OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+        ps.print(geometry.asGeoJson());
+    };
+
+    @Override
+    public IPrinter createPrinter() {
+        return PRINTER;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
index 5cae68c..27fc7eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
@@ -113,6 +113,9 @@
             case UUID:
                 AUUIDPrinterFactory.PRINTER.print(b, s, l, ps);
                 return true;
+            case GEOMETRY:
+                AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
             default:
                 return false;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
new file mode 100644
index 0000000..0a74ab7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.esri.core.geometry.OperatorImportFromWkb;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WkbImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AGeometrySerializerDeserializer implements ISerializerDeserializer<AGeometry> {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AGeometrySerializerDeserializer INSTANCE = new AGeometrySerializerDeserializer();
+
+    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+
+    private AGeometrySerializerDeserializer() {
+    }
+
+    @Override
+    public AGeometry deserialize(DataInput in) throws HyracksDataException {
+        try {
+            int length = in.readInt();
+            byte[] bytes = new byte[length];
+            in.readFully(bytes);
+            ByteBuffer buffer = ByteBuffer.wrap(bytes);
+            OGCGeometry geometry = OGCGeometry.createFromOGCStructure(
+                    OperatorImportFromWkb.local().executeOGC(WkbImportFlags.wkbImportDefaults, buffer, null),
+                    DEFAULT_CRS);
+            return new AGeometry(geometry);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public void serialize(AGeometry instance, DataOutput out) throws HyracksDataException {
+        try {
+            OGCGeometry geometry = instance.getGeometry();
+            byte[] buffer = geometry.asBinary().array();
+            // For efficiency, we store the size of the geometry in bytes in the first 32 bits
+            // This allows AsterixDB to skip over this attribute if needed.
+            out.writeInt(buffer.length);
+            out.write(buffer);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index aef4ca1..57f3449 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.ADuration;
 import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
@@ -120,6 +121,8 @@
                 return AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
             case MULTISET:
                 return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
+            case GEOMETRY:
+                return AGeometrySerializerDeserializer.INSTANCE.deserialize(in);
             default:
                 throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
         }
@@ -213,6 +216,9 @@
             case TYPE:
                 ATypeSerializerDeserializer.INSTANCE.serialize((IAType) instance, out);
                 break;
+            case GEOMETRY:
+                AGeometrySerializerDeserializer.INSTANCE.serialize((AGeometry) instance, out);
+                break;
             default:
                 throw new HyracksDataException(
                         "No serializer/deserializer implemented for type " + t.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
index 888b34c..4bb9f08 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
@@ -69,4 +69,8 @@
             throw HyracksDataException.create(e);
         }
     }
-}
+
+    public void serialize(char[] buffer, int start, int length, DataOutput out) throws IOException {
+        UTF8StringUtil.writeUTF8(buffer, start, length, out, utf8StringWriter);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 678a864..6906ae4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IDataFormat {
     public ISerializerDeserializerProvider getSerdeProvider();
@@ -66,16 +67,17 @@
     public IMissingWriterFactory getMissingWriterFactory();
 
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException;
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName, SourceLocation sourceLoc)
+            throws AlgebricksException;
 
     public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn) throws AlgebricksException;
+            List<String> fldName, int recordColumn, SourceLocation sourceLoc) throws AlgebricksException;
 
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException;
 
     public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
-            throws AlgebricksException;
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR,
+            SourceLocation sourceLoc) throws AlgebricksException;
 
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
index fd4df0d..6b33408 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
@@ -18,11 +18,14 @@
  */
 package org.apache.asterix.formats.nontagged;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class BinaryBooleanInspector implements IBinaryBooleanInspector {
     private static final BinaryBooleanInspector INSTANCE = new BinaryBooleanInspector();
@@ -36,11 +39,13 @@
         }
     };
 
+    private static final String NAME = "boolean-inspector";
+
     private BinaryBooleanInspector() {
     }
 
     @Override
-    public boolean getBooleanValue(byte[] bytes, int offset, int length) {
+    public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException {
         byte serializedTypeTag = bytes[offset];
         if (serializedTypeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                 || serializedTypeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
@@ -49,8 +54,9 @@
         /** check if the runtime type is boolean */
         ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serializedTypeTag);
         if (typeTag != ATypeTag.BOOLEAN) {
-            throw new IllegalStateException("Runtime error: the select condition should be of the boolean type!");
+            throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH, NAME, 0, ATypeTag.BOOLEAN, typeTag);
         }
+
         return bytes[offset + 1] == 1;
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
index 81dc0c2..d66673d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
@@ -162,6 +163,8 @@
                 return AUUIDSerializerDeserializer.INSTANCE;
             case SHORTWITHOUTTYPEINFO:
                 return ShortSerializerDeserializer.INSTANCE;
+            case GEOMETRY:
+                return AGeometrySerializerDeserializer.INSTANCE;
             default:
                 throw new NotImplementedException(
                         "No serializer/deserializer implemented for type " + aqlType.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
new file mode 100644
index 0000000..3b9c55d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.base;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+import java.io.IOException;
+
+public class AGeometry implements IAObject {
+
+    protected OGCGeometry geometry;
+
+    public AGeometry(OGCGeometry geometry) {
+        this.geometry = geometry;
+    }
+
+    public OGCGeometry getGeometry() {
+        return geometry;
+    }
+
+    @Override
+    public IAType getType() {
+        return BuiltinType.AGEOMETRY;
+    }
+
+    @Override
+    public boolean deepEqual(IAObject obj) {
+        if (!(obj instanceof AGeometry)) {
+            return false;
+        } else {
+            AGeometry p = (AGeometry) obj;
+            return p.geometry.equals(geometry);
+        }
+    }
+
+    @Override
+    public int hash() {
+        return geometry.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return geometry.toString();
+    }
+
+    @Override
+    public ObjectNode toJSON() {
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode json = null;
+        try {
+            json = (ObjectNode) om.readTree(geometry.asGeoJson());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return json;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
new file mode 100644
index 0000000..346d68a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.base;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AMutableGeometry extends AGeometry {
+
+    private OperatorImportFromWkt wktImporter;
+
+    public AMutableGeometry(OGCGeometry geom) {
+        super(geom);
+        wktImporter = OperatorImportFromWkt.local();
+    }
+
+    public void setValue(OGCGeometry geom) {
+        this.geometry = geom;
+    }
+
+    public void parseWKT(String wkt) {
+        OGCStructure structure;
+
+        structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, wkt, null);
+        this.geometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
index 2fb69ab..9a097dc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
@@ -23,15 +23,16 @@
 
 public class AMutableUUID extends AUUID {
 
-    public void parseUUIDString(String tokenImage) throws HyracksDataException {
-        if (tokenImage.length() != UUID_CHARS) {
-            throw new HyracksDataException("This is not a correct UUID value: " + tokenImage);
+    private final byte[] hexBytesBuffer = new byte[UUID_CHARS];
+
+    public void parseUUIDString(char[] buffer, int begin, int len) throws HyracksDataException {
+        if (len != UUID_CHARS) {
+            throw new HyracksDataException("This is not a correct UUID value: " + new String(buffer, begin, len));
         }
-        byte[] hexBytes = new byte[UUID_CHARS];
-        for (int i = 0; i < tokenImage.length(); i++) {
-            hexBytes[i] = (byte) tokenImage.charAt(i);
+        for (int i = 0; i < len; i++) {
+            hexBytesBuffer[i] = (byte) buffer[begin + i];
         }
-        parseUUIDHexBytes(hexBytes, 0);
+        parseUUIDHexBytes(hexBytesBuffer, 0);
     }
 
     public void parseUUIDHexBytes(byte[] serString, int offset) throws HyracksDataException {
@@ -122,4 +123,4 @@
         }
     }
 
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
index 291d963..39f5b3a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
@@ -63,7 +63,6 @@
      * @throws HyracksDataException
      */
     public static int parseTimePart(String timeString, int start, int length) throws HyracksDataException {
-
         int offset = 0;
 
         int hour = 0, min = 0, sec = 0, millis = 0;
@@ -518,4 +517,4 @@
         return timezone;
     }
 
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
index f837e36..98abb97 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -148,20 +148,20 @@
     };
 
     private static final byte[][] TIMEZONE_IDS;
-    private static final int[] TIMEZONE_OFFSETS;
+    private static final TimeZone[] TIMEZONE_VALUES;
 
     static {
         String[] tzIds = TimeZone.getAvailableIDs();
         int tzCount = tzIds.length;
         TIMEZONE_IDS = new byte[tzCount][];
-        TIMEZONE_OFFSETS = new int[tzCount];
+        TIMEZONE_VALUES = new TimeZone[tzCount];
 
         for (int i = 0; i < tzCount; i++) {
             TIMEZONE_IDS[i] = tzIds[i].getBytes(ENCODING);
         }
         Arrays.sort(TIMEZONE_IDS, byteArrayComparator);
         for (int i = 0; i < tzCount; i++) {
-            TIMEZONE_OFFSETS[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING)).getRawOffset();
+            TIMEZONE_VALUES[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING));
         }
     }
 
@@ -241,9 +241,10 @@
         return -1;
     }
 
-    private int binaryTimezoneIDSearch(byte[] barray, int start, int length) {
-        return Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
+    public static TimeZone findTimeZone(byte[] barray, int start, int length) {
+        int idx = Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
                 Arrays.copyOfRange(barray, start, start + length), byteArrayComparator);
+        return idx >= 0 ? TIMEZONE_VALUES[idx] : null;
     }
 
     private int indexOf(byte[] barray, int start, int length, char c) {
@@ -683,10 +684,10 @@
                                 || data[dataStart + timezoneEndField] == '_')) {
                             timezoneEndField++;
                         }
-                        int searchIdx = binaryTimezoneIDSearch(data, dataStart + dataStringPointer,
-                                timezoneEndField - dataStringPointer);
-                        if (searchIdx >= 0) {
-                            timezone = TIMEZONE_OFFSETS[searchIdx];
+                        TimeZone tz =
+                                findTimeZone(data, dataStart + dataStringPointer, timezoneEndField - dataStringPointer);
+                        if (tz != null) {
+                            timezone = tz.getRawOffset();
                         } else {
                             if (raiseParseDataError) {
                                 throw new AsterixTemporalTypeParseException("Unexpected timezone string: " + new String(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
index 71a5649..1a5d5c6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
@@ -23,6 +23,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IncompatibleTypeException extends CompilationException {
 
@@ -32,7 +33,19 @@
     }
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag typeTagLeft,
+            ATypeTag typeTagRight) {
+        super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, sourceLoc, fid.getName(), typeTagLeft, typeTagRight);
+    }
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
     public IncompatibleTypeException(String functionName, ATypeTag typeTagLeft, ATypeTag typeTagRight) {
         super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, functionName, typeTagLeft, typeTagRight);
     }
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, String functionName, ATypeTag typeTagLeft,
+            ATypeTag typeTagRight) {
+        super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, sourceLoc, functionName, typeTagLeft, typeTagRight);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
index 699629b..099930f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class InvalidExpressionException extends CompilationException {
 
@@ -36,9 +37,21 @@
                 toExpectedTypeString(exprKinds));
     }
 
+    public InvalidExpressionException(SourceLocation sourceLoc, FunctionIdentifier fid, int index,
+            ILogicalExpression actualExpr, LogicalExpressionTag... exprKinds) {
+        super(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, fid.getName(), indexToPosition(index),
+                actualExpr.toString(), toExpectedTypeString(exprKinds));
+    }
+
     public InvalidExpressionException(String functionName, int index, ILogicalExpression actualExpr,
             LogicalExpressionTag... exprKinds) {
         super(ErrorCode.COMPILATION_INVALID_EXPRESSION, functionName, indexToPosition(index), actualExpr.toString(),
                 toExpectedTypeString(exprKinds));
     }
+
+    public InvalidExpressionException(SourceLocation sourceLoc, String functionName, int index,
+            ILogicalExpression actualExpr, LogicalExpressionTag... exprKinds) {
+        super(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, functionName, indexToPosition(index),
+                actualExpr.toString(), toExpectedTypeString(exprKinds));
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
index 7218a3a..9dab8ac 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeMismatchException extends CompilationException {
 
@@ -37,9 +38,22 @@
     }
 
     // Parameter type mistmatch.
+    public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, ATypeTag actualTypeTag,
+            ATypeTag... expectedTypeTags) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH, sourceLoc, fid.getName(), indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags), actualTypeTag);
+    }
+
+    // Parameter type mistmatch.
     public TypeMismatchException(String functionName, Integer i, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
         super(ErrorCode.COMPILATION_TYPE_MISMATCH, functionName, indexToPosition(i),
                 toExpectedTypeString(expectedTypeTags), actualTypeTag);
     }
 
+    // Parameter type mistmatch.
+    public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, ATypeTag actualTypeTag,
+            ATypeTag... expectedTypeTags) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH, sourceLoc, functionName, indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags), actualTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
index 0393887..06491f6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
@@ -23,6 +23,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedItemTypeException extends CompilationException {
 
@@ -32,7 +33,17 @@
     }
 
     // Unsupported item type.
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag itemTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_ITEM, sourceLoc, fid.getName(), itemTypeTag);
+    }
+
+    // Unsupported item type.
     public UnsupportedItemTypeException(String functionName, ATypeTag itemTypeTag) {
         super(ErrorCode.COMPILATION_TYPE_ITEM, functionName, itemTypeTag);
     }
+
+    // Unsupported item type.
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, String functionName, ATypeTag itemTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_ITEM, sourceLoc, functionName, itemTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
index bc69a48..5441cf4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
@@ -23,6 +23,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedTypeException extends CompilationException {
 
@@ -32,7 +33,17 @@
     }
 
     // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag actualTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, fid.getName(), actualTypeTag);
+    }
+
+    // Unsupported input type.
     public UnsupportedTypeException(String funcName, ATypeTag actualTypeTag) {
         super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, funcName, actualTypeTag);
     }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, String funcName, ATypeTag actualTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, funcName, actualTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
index 35440e4..3aea6dc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
@@ -18,25 +18,29 @@
  */
 package org.apache.asterix.om.functions;
 
-import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractFunctionDescriptor implements IFunctionDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    @Override
-    public abstract FunctionIdentifier getIdentifier();
+    protected SourceLocation sourceLoc;
 
     @Override
-    public abstract FunctionDescriptorTag getFunctionDescriptorTag();
+    public void setImmutableStates(Object... states) {
+    }
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
@@ -66,5 +70,4 @@
             throws AlgebricksException {
         throw new NotImplementedException("Not Implemented: " + getIdentifier());
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 8cd18fc..a80ba28 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.om.typecomputer.impl.ADoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ADurationTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AFloatTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AGeometryTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt16TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt32TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt64TypeComputer;
@@ -61,23 +62,27 @@
 import org.apache.asterix.om.typecomputer.impl.CollectionMemberResultType;
 import org.apache.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.DoubleIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByNameResultType;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessNestedResultType;
 import org.apache.asterix.om.typecomputer.impl.FullTextContainsResultTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.GetOverlappingInvervalTypeComputer;
-import org.apache.asterix.om.typecomputer.impl.IfNanOrInfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfMissingOrNullTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfMissingTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.IfNanOrInfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfNullTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.LocalAvgTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.MinMaxAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.MissingIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NonTaggedGetItemResultType;
 import org.apache.asterix.om.typecomputer.impl.NotUnknownTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NullIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericAddSubMulDivTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NumericDivideTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericDoubleOutputFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericInt8OutputFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericRound2TypeComputer;
@@ -213,6 +218,14 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-names", 1);
     public static final FunctionIdentifier RECORD_PAIRS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-pairs", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier GEOMETRY_CONSTRUCTOR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-geojson", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RECORD_REMOVE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-remove", 2);
+    public static final FunctionIdentifier RECORD_RENAME =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-rename", 3);
+    public static final FunctionIdentifier RECORD_UNWRAP =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-unwrap", 1);
 
     // numeric
     public static final FunctionIdentifier NUMERIC_UNARY_MINUS =
@@ -225,9 +238,10 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-divide", 2);
     public static final FunctionIdentifier NUMERIC_MOD =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-mod", 2);
-    public static final FunctionIdentifier NUMERIC_IDIV =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-idiv", 2);
-    public static final FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
+    public static final FunctionIdentifier NUMERIC_DIV =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-div", 2);
+    public static final FunctionIdentifier NUMERIC_POWER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "power", 2);
     public static final FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "abs", 1);
     public static final FunctionIdentifier NUMERIC_ACOS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "acos", 1);
@@ -237,6 +251,10 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan", 1);
     public static final FunctionIdentifier NUMERIC_ATAN2 =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan2", 2);
+    public static final FunctionIdentifier NUMERIC_DEGREES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "degrees", 1);
+    public static final FunctionIdentifier NUMERIC_RADIANS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "radians", 1);
     public static final FunctionIdentifier NUMERIC_COS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cos", 1);
     public static final FunctionIdentifier NUMERIC_SIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sin", 1);
     public static final FunctionIdentifier NUMERIC_TAN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tan", 1);
@@ -247,6 +265,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sqrt", 1);
     public static final FunctionIdentifier NUMERIC_SIGN =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sign", 1);
+    public static final FunctionIdentifier NUMERIC_E = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "e", 0);
+    public static final FunctionIdentifier NUMERIC_PI = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "pi", 0);
 
     public static final FunctionIdentifier NUMERIC_CEILING =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ceiling", 1);
@@ -321,6 +341,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 3);
     public static final FunctionIdentifier STRING_REPLACE_WITH_LIMIT =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 4);
+    public static final FunctionIdentifier STRING_REVERSE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reverse", 1);
     public static final FunctionIdentifier STRING_LENGTH =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-length", 1);
     public static final FunctionIdentifier STRING_LIKE =
@@ -733,6 +755,129 @@
     public static final FunctionIdentifier UUID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "uuid", 0);
     public static final FunctionIdentifier CREATE_QUERY_UID =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-query-uid", 0);
+    public static final FunctionIdentifier RANDOM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 0);
+    public static final FunctionIdentifier RANDOM_WITH_SEED =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 1);
+
+    //Geo
+    public static final FunctionIdentifier ST_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-area", 1);
+    public static final FunctionIdentifier ST_MAKE_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 2);
+    public static final FunctionIdentifier ST_MAKE_POINT3D =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 3);
+    public static final FunctionIdentifier ST_MAKE_POINT3D_M =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 4);
+    public static final FunctionIdentifier ST_INTERSECTS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersects", 2);
+    public static final FunctionIdentifier ST_UNION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 2);
+    public static final FunctionIdentifier ST_IS_COLLECTION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-collection", 1);
+    public static final FunctionIdentifier ST_CONTAINS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-contains", 2);
+    public static final FunctionIdentifier ST_CROSSES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-crosses", 2);
+    public static final FunctionIdentifier ST_DISJOINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-disjoint", 2);
+    public static final FunctionIdentifier ST_EQUALS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-equals", 2);
+    public static final FunctionIdentifier ST_OVERLAPS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-overlaps", 2);
+    public static final FunctionIdentifier ST_TOUCHES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-touches", 2);
+    public static final FunctionIdentifier ST_WITHIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-within", 2);
+    public static final FunctionIdentifier ST_IS_EMPTY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-empty", 1);
+    public static final FunctionIdentifier ST_IS_SIMPLE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-simple", 1);
+    public static final FunctionIdentifier ST_COORD_DIM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-coord-dim", 1);
+    public static final FunctionIdentifier ST_DIMENSION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-dimension", 1);
+    public static final FunctionIdentifier GEOMETRY_TYPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "geometry-type", 1);
+    public static final FunctionIdentifier ST_M = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-m", 1);
+    public static final FunctionIdentifier ST_N_RINGS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-rings", 1);
+    public static final FunctionIdentifier ST_N_POINTS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-points", 1);
+    public static final FunctionIdentifier ST_NUM_GEOMETRIIES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-geometries", 1);
+    public static final FunctionIdentifier ST_NUM_INTERIOR_RINGS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-interior-rings", 1);
+    public static final FunctionIdentifier ST_SRID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-srid", 1);
+    public static final FunctionIdentifier ST_X = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x", 1);
+    public static final FunctionIdentifier ST_Y = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y", 1);
+    public static final FunctionIdentifier ST_X_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-max", 1);
+    public static final FunctionIdentifier ST_X_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-min", 1);
+    public static final FunctionIdentifier ST_Y_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-max", 1);
+    public static final FunctionIdentifier ST_Y_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-min", 1);
+    public static final FunctionIdentifier ST_Z = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z", 1);
+    public static final FunctionIdentifier ST_Z_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-min", 1);
+    public static final FunctionIdentifier ST_Z_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-max", 1);
+    public static final FunctionIdentifier ST_AS_BINARY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-binary", 1);
+    public static final FunctionIdentifier ST_AS_TEXT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-text", 1);
+    public static final FunctionIdentifier ST_AS_GEOJSON =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-geojson", 1);
+    public static final FunctionIdentifier ST_DISTANCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-distance", 2);
+    public static final FunctionIdentifier ST_LENGTH =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-length", 1);
+    public static final FunctionIdentifier SCALAR_ST_UNION_AGG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 1);
+    public static final FunctionIdentifier ST_UNION_AGG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union-agg", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_TEXT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_TEXT_SRID =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 2);
+    public static final FunctionIdentifier ST_GEOM_FROM_WKB =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_WKB_SRID =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 2);
+    public static final FunctionIdentifier ST_LINE_FROM_MULTIPOINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-line-from-multipoint", 1);
+    public static final FunctionIdentifier ST_MAKE_ENVELOPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-envelope", 5);
+    public static final FunctionIdentifier ST_IS_CLOSED =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-closed", 1);
+    public static final FunctionIdentifier ST_IS_RING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-ring", 1);
+    public static final FunctionIdentifier ST_RELATE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-relate", 3);
+    public static final FunctionIdentifier ST_BOUNDARY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-boundary", 1);
+    public static final FunctionIdentifier ST_END_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-end-point", 1);
+    public static final FunctionIdentifier ST_ENVELOPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-envelope", 1);
+    public static final FunctionIdentifier ST_EXTERIOR_RING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-exterior-ring", 1);
+    public static final FunctionIdentifier ST_GEOMETRY_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geometry-n", 2);
+    public static final FunctionIdentifier ST_INTERIOR_RING_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-interior-ring-n", 2);
+    public static final FunctionIdentifier ST_POINT_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-point-n", 2);
+    public static final FunctionIdentifier ST_START_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-start-point", 1);
+    public static final FunctionIdentifier ST_DIFFERENCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-difference", 2);
+    public static final FunctionIdentifier ST_INTERSECTION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersection", 2);
+    public static final FunctionIdentifier ST_SYM_DIFFERENCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-sym-difference", 2);
+    public static final FunctionIdentifier ST_POLYGONIZE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-polygonize", 1);
 
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR =
@@ -876,6 +1021,15 @@
     public static final FunctionIdentifier IF_NAN_OR_INF =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "if-nan-or-inf", FunctionIdentifier.VARARGS);
 
+    public static final FunctionIdentifier MISSING_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "missing-if", 2);
+    public static final FunctionIdentifier NULL_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "null-if", 2);
+    public static final FunctionIdentifier NAN_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nan-if", 2);
+    public static final FunctionIdentifier POSINF_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "posinf-if", 2);
+    public static final FunctionIdentifier NEGINF_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "neginf-if", 2);
+
     public static final FunctionIdentifier TO_ATOMIC =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-atomic", 1);
     public static final FunctionIdentifier TO_ARRAY =
@@ -944,7 +1098,6 @@
         addPrivateFunction(CHECK_UNKNOWN, NotUnknownTypeComputer.INSTANCE, true);
         addPrivateFunction(ANY_COLLECTION_MEMBER, CollectionMemberResultType.INSTANCE, true);
         addFunction(BOOLEAN_CONSTRUCTOR, ABooleanTypeComputer.INSTANCE, true);
-        addFunction(CARET, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addFunction(CIRCLE_CONSTRUCTOR, ACircleTypeComputer.INSTANCE, true);
         addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE, true);
 
@@ -960,6 +1113,8 @@
         addFunction(UUID, AUUIDTypeComputer.INSTANCE, false);
         addFunction(CREATE_QUERY_UID, ABinaryTypeComputer.INSTANCE, false);
         addFunction(UUID_CONSTRUCTOR, AUUIDTypeComputer.INSTANCE, true);
+        addFunction(RANDOM, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(RANDOM_WITH_SEED, ADoubleTypeComputer.INSTANCE, false);
 
         addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE, true);
         addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE, true);
@@ -981,9 +1136,6 @@
         addPrivateFunction(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
         addPrivateFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
         addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
-        addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
-        addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
-        addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
         addPrivateFunction(INDEX_SEARCH, AnyTypeComputer.INSTANCE, true);
         addFunction(INT8_CONSTRUCTOR, AInt8TypeComputer.INSTANCE, true);
         addFunction(INT16_CONSTRUCTOR, AInt16TypeComputer.INSTANCE, true);
@@ -998,20 +1150,25 @@
         addPrivateFunction(NUMERIC_UNARY_MINUS, UnaryMinusTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_MULTIPLY, NumericAddSubMulDivTypeComputer.INSTANCE, true);
-        addPrivateFunction(NUMERIC_DIVIDE, NumericAddSubMulDivTypeComputer.INSTANCE, true);
+        addPrivateFunction(NUMERIC_DIVIDE, NumericDivideTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_MOD, NumericAddSubMulDivTypeComputer.INSTANCE, true);
-        addPrivateFunction(NUMERIC_IDIV, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(NUMERIC_DIV, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ABS, NumericUnaryFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ACOS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ASIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ATAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ATAN2, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_DEGREES, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_RADIANS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_COS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_TAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_E, ADoubleTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_EXP, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_LN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_LOG, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_PI, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_POWER, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SQRT, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SIGN, NumericInt8OutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_CEILING, NumericUnaryFunctionTypeComputer.INSTANCE, true);
@@ -1020,9 +1177,6 @@
         addFunction(NUMERIC_ROUND_HALF_TO_EVEN, NumericUnaryFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ROUND_HALF_TO_EVEN2, NumericRound2TypeComputer.INSTANCE, true);
         addFunction(NUMERIC_TRUNC, NumericRound2TypeComputer.INSTANCE, true);
-        addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
-        addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
-        addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
 
         addFunction(BINARY_LENGTH, UnaryBinaryInt64TypeComputer.INSTANCE, true);
         addFunction(PARSE_BINARY, ABinaryTypeComputer.INSTANCE, true);
@@ -1063,6 +1217,7 @@
         addFunction(STRING_REGEXP_REPLACE_WITH_FLAG, StringStringTypeComputer.INSTANCE, true);
         addFunction(STRING_REPLACE, StringStringTypeComputer.INSTANCE, true);
         addFunction(STRING_REPLACE_WITH_LIMIT, StringIntToStringTypeComputer.INSTANCE_TRIPLE_STRING, true);
+        addFunction(STRING_REVERSE, StringStringTypeComputer.INSTANCE, true);
         addFunction(SUBSTRING_BEFORE, StringStringTypeComputer.INSTANCE, true);
         addFunction(SUBSTRING_AFTER, StringStringTypeComputer.INSTANCE, true);
         addPrivateFunction(STRING_EQUAL, StringBooleanTypeComputer.INSTANCE, true);
@@ -1087,6 +1242,19 @@
         addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
         addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
 
+        addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+        addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
+        addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
+        addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
+        addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
+        addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+
+        addFunction(MISSING_IF, MissingIfTypeComputer.INSTANCE, true);
+        addFunction(NULL_IF, NullIfTypeComputer.INSTANCE, true);
+        addFunction(NAN_IF, DoubleIfTypeComputer.INSTANCE, true);
+        addFunction(POSINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+        addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+
         // Aggregate Functions
         addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1198,6 +1366,71 @@
         addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE, true);
         addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE, true);
 
+        //geo functions
+        addFunction(ST_AREA, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT3D, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT3D_M, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERSECTS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_UNION, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_CROSSES, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_DISJOINT, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_EQUALS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_OVERLAPS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_TOUCHES, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_WITHIN, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_EMPTY, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_SIMPLE, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_COORD_DIM, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_DIMENSION, AInt32TypeComputer.INSTANCE, true);
+        addFunction(GEOMETRY_TYPE, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_M, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_N_RINGS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_N_POINTS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_NUM_GEOMETRIIES, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_NUM_INTERIOR_RINGS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_SRID, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_X, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_X_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_X_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_BINARY, ABinaryTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_TEXT, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_GEOJSON, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_DISTANCE, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_LENGTH, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_TEXT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_TEXT_SRID, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_WKB, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_WKB_SRID, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_LINE_FROM_MULTIPOINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_CLOSED, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_RING, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_RELATE, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_BOUNDARY, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_END_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_EXTERIOR_RING, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOMETRY_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERIOR_RING_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_POINT_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_START_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERSECTION, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_SYM_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+        addPrivateFunction(ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_POLYGONIZE, AGeometryTypeComputer.INSTANCE, true);
+
         // Binary functions
         addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
         addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
@@ -1232,6 +1465,10 @@
         addFunction(RECORD_LENGTH, AInt64TypeComputer.INSTANCE_NULLABLE, true);
         addFunction(RECORD_NAMES, OrderedListOfAStringTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(RECORD_PAIRS, RecordPairsTypeComputer.INSTANCE, true);
+        addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(RECORD_REMOVE, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_RENAME, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_UNWRAP, AnyTypeComputer.INSTANCE, true);
 
         // temporal type accessors
         addFunction(ACCESSOR_TEMPORAL_YEAR, AInt64TypeComputer.INSTANCE, true);
@@ -1563,6 +1800,13 @@
 
         addDistinctAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM);
         addScalarAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM_DISTINCT);
+
+        // SPATIAL AGGREGATES
+
+        addAgg(ST_UNION_AGG);
+        addLocalAgg(ST_UNION_AGG, ST_UNION_AGG);
+        addIntermediateAgg(ST_UNION_AGG, ST_UNION_AGG);
+        addGlobalAgg(ST_UNION_AGG, ST_UNION_AGG);
     }
 
     static {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
index d9bb73a..1fc9ef9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
@@ -28,11 +28,13 @@
 import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IFunctionDescriptor extends Serializable {
 
-    default void setImmutableStates(Object... states) {
-    }
+    void setImmutableStates(Object... states);
+
+    void setSourceLocation(SourceLocation sourceLoc);
 
     FunctionIdentifier getIdentifier();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
index 4d6b7e6..ae7c996 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This abstract class takes care of the handling of optional types.
@@ -39,23 +40,26 @@
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
         AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
         String funcName = functionCallExpression.getFunctionIdentifier().getName();
-        return TypeComputeUtils.resolveResultType(expression, env, (index, type) -> checkArgType(funcName, index, type),
-                this::getResultType, propagateNullAndMissing());
+        return TypeComputeUtils.resolveResultType(expression, env,
+                (index, type, sourceLoc) -> checkArgType(funcName, index, type, sourceLoc), this::getResultType,
+                propagateNullAndMissing());
     }
 
     /**
      * Checks whether an input type violates the requirement.
      *
-     * @param funcName
-     *            the function name.
      * @param argIndex,
      *            the index of the argument to consider.
      * @param type,
      *            the type of the input argument.
+     * @param funcName
+     *            the function name.
+     * @param sourceLoc
+     *            the source location
      * @throws AlgebricksException
      */
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
-
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
     }
 
     /**
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
index 8c40903..7fa987f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
@@ -44,7 +44,7 @@
             ATypeTag actualTypeTag = TypeComputeUtils.getActualType(inputType).getTypeTag();
             if (!ATypeHierarchy.isCompatible(requiredTypeTag, actualTypeTag)) {
                 String funcName = expr.getFunctionIdentifier().getName();
-                throw new IncompatibleTypeException(funcName, actualTypeTag, requiredTypeTag);
+                throw new IncompatibleTypeException(expr.getSourceLocation(), funcName, actualTypeTag, requiredTypeTag);
             }
             expr.setOpaqueParameters(opaqueParameters);
             changed = true;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java
new file mode 100644
index 0000000..e85410ab5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class AGeometryTypeComputer extends AbstractResultTypeComputer {
+
+    public static final AGeometryTypeComputer INSTANCE = new AGeometryTypeComputer();
+
+    private AGeometryTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return BuiltinType.AGEOMETRY;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
index 2626c6b..1d2cb21 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
@@ -33,12 +33,14 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 abstract class AbstractIfMissingOrNullTypeComputer implements IResultTypeComputer {
     @Override
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
         AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        SourceLocation sourceLoc = fce.getSourceLocation();
         IAType outPrimeType = null;
         ATypeTag outQuantifier = null; // could be 'missing' or 'null'
 
@@ -66,22 +68,23 @@
                     } else {
                         IAType primeType = getOutputPrimeType(unionType);
                         ATypeTag quantifier = outQuantifier != null ? outQuantifier : getOutputQuantifier(unionType);
-                        return createOutputType(TypeResolverUtil.resolve(outPrimeType, primeType), quantifier);
+                        return createOutputType(TypeResolverUtil.resolve(outPrimeType, primeType), quantifier,
+                                sourceLoc);
                     }
                 }
             } else {
                 // ANY or no intersection
                 return outPrimeType == null ? argType
-                        : createOutputType(TypeResolverUtil.resolve(outPrimeType, argType), outQuantifier);
+                        : createOutputType(TypeResolverUtil.resolve(outPrimeType, argType), outQuantifier, sourceLoc);
             }
         }
 
         if (outPrimeType == null) {
             return BuiltinType.ANULL;
         }
-        IAType outType = createOutputType(outPrimeType, ATypeTag.NULL);
+        IAType outType = createOutputType(outPrimeType, ATypeTag.NULL, sourceLoc);
         if (outQuantifier == ATypeTag.MISSING) {
-            outType = createOutputType(outType, ATypeTag.MISSING);
+            outType = createOutputType(outType, ATypeTag.MISSING, sourceLoc);
         }
         return outType;
     }
@@ -96,7 +99,8 @@
         return type.getActualType();
     }
 
-    private IAType createOutputType(IAType primeType, ATypeTag quantifier) throws AlgebricksException {
+    private IAType createOutputType(IAType primeType, ATypeTag quantifier, SourceLocation sourceLoc)
+            throws CompilationException {
         if (quantifier == null || primeType.getTypeTag() == ATypeTag.ANY) {
             return primeType;
         }
@@ -106,7 +110,8 @@
             case NULL:
                 return AUnionType.createNullableType(primeType, null);
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, String.valueOf(quantifier));
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                        "Unexpected quantifier: " + quantifier);
         }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
index 27f364a..08d2ca5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
@@ -24,14 +24,16 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 abstract public class AbstractStringTypeComputer extends AbstractResultTypeComputer {
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (actualTypeTag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
index 80ac8ef..c6c744c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
@@ -41,8 +41,8 @@
         AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
         String funcName = functionCallExpression.getFunctionIdentifier().getName();
         // Boolean type computer doesn't follow the null/missing-in/out semantics.
-        return TypeComputeUtils.resolveResultType(expression, env, (index, type) -> checkArgType(funcName, index, type),
-                this::getResultType, false);
+        return TypeComputeUtils.resolveResultType(expression, env,
+                (index, type, sourceLoc) -> checkArgType(funcName, index, type, sourceLoc), this::getResultType, false);
     }
 
     @Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
index b328cf4..d5e576e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
@@ -71,7 +71,8 @@
             fieldTypes[i] = e2Type;
             fieldNames[i] = ConstantExpressionUtil.getStringConstant(e1);
             if (fieldNames[i] == null) {
-                throw new InvalidExpressionException(funcName, 2 * i, e1, LogicalExpressionTag.CONSTANT);
+                throw new InvalidExpressionException(f.getSourceLocation(), funcName, 2 * i, e1,
+                        LogicalExpressionTag.CONSTANT);
             }
             i++;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
index cc2f9b6..9e88438 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CollectionMemberResultType extends AbstractResultTypeComputer {
     public static final CollectionMemberResultType INSTANCE = new CollectionMemberResultType();
@@ -34,10 +35,12 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (type.getTypeTag() != ATypeTag.MULTISET && type.getTypeTag() != ATypeTag.ARRAY) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.MULTISET,
+                    ATypeTag.ARRAY);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java
new file mode 100644
index 0000000..105ea52
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class DoubleIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final DoubleIfTypeComputer INSTANCE = new DoubleIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.DOUBLE ? inputType : BuiltinType.ANY;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
index 73c019c..71fa15c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessByIndexResultType extends AbstractResultTypeComputer {
 
@@ -37,13 +38,14 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1 && actualTypeTag != ATypeTag.INTEGER) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
index e9af1ab..948f8d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessByNameResultType extends AbstractResultTypeComputer {
 
@@ -37,13 +38,14 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1 && actualTypeTag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
index 3eae173..20f51ec 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessNestedResultType extends AbstractResultTypeComputer {
     public static final FieldAccessNestedResultType INSTANCE = new FieldAccessNestedResultType();
@@ -46,29 +47,31 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1) {
             switch (actualTypeTag) {
                 case STRING:
                     break;
                 case ARRAY:
-                    checkOrderedList(funcName, type);
+                    checkOrderedList(funcName, type, sourceLoc);
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.ARRAY);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                            ATypeTag.ARRAY);
             }
         }
     }
 
-    private void checkOrderedList(String funcName, IAType type) throws AlgebricksException {
+    private void checkOrderedList(String funcName, IAType type, SourceLocation sourceLoc) throws AlgebricksException {
         AOrderedListType listType = (AOrderedListType) type;
         ATypeTag itemTypeTag = listType.getItemType().getTypeTag();
         if (itemTypeTag != ATypeTag.STRING && itemTypeTag != ATypeTag.ANY) {
-            throw new UnsupportedItemTypeException(funcName, itemTypeTag);
+            throw new UnsupportedItemTypeException(sourceLoc, funcName, itemTypeTag);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
index 0748745..606a68e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FullTextContainsResultTypeComputer extends AbstractResultTypeComputer {
 
@@ -34,24 +35,26 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         // Expression1 should be a string.
         if (argIndex == 0 && actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.ANY) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING);
         }
         // Expression2 should be a string, or an (un)ordered list.
         if (argIndex == 1 && (actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.MULTISET
                 && actualTypeTag != ATypeTag.ARRAY && actualTypeTag != ATypeTag.ANY)) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.MULTISET,
-                    ATypeTag.ARRAY);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                    ATypeTag.MULTISET, ATypeTag.ARRAY);
         }
         // Each option name should be a string if it is already processed by FullTextContainsParameterCheckRule.
         // Before, the third argument should be a record if exists.
         // The structure is: arg2 = optionName1, arg3 = optionValue1, arg4 = optionName1, arg5 = optionValue2, ...
         if (argIndex > 1 && argIndex % 2 == 0 && (actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.OBJECT
                 && actualTypeTag != ATypeTag.ANY)) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                    ATypeTag.OBJECT);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
index cc19ac4..51fd6fc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
@@ -24,12 +24,13 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * The first argument of INJECT_FAILURE can be any data model instance and will be passed verbatim to the
  * caller. The second argument is a boolean that determines if the invocation throws an exception.
  *
- * Consequently {@link #checkArgType(String, int, IAType)} validates that the second argument is a
+ * Consequently {@link AbstractResultTypeComputer#checkArgType(String, int, IAType, SourceLocation)} validates that the second argument is a
  * boolean and {@link #getResultType(ILogicalExpression, IAType...)} returns the type of the first
  * argument.
  */
@@ -38,9 +39,10 @@
     public static final InjectFailureTypeComputer INSTANCE = new InjectFailureTypeComputer();
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (argIndex == 1 && type.getTypeTag() != ATypeTag.BOOLEAN) {
-            throw new TypeMismatchException(funcName, argIndex, type.getTypeTag(), ATypeTag.BOOLEAN);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, type.getTypeTag(), ATypeTag.BOOLEAN);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
index 0526e7d..abdff7e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class MinMaxAggTypeComputer extends AbstractResultTypeComputer {
 
@@ -34,7 +35,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case DOUBLE:
@@ -52,7 +54,7 @@
             case ANY:
                 return;
             default:
-                throw new UnsupportedTypeException(funcName, tag);
+                throw new UnsupportedTypeException(sourceLoc, funcName, tag);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java
new file mode 100644
index 0000000..bb5203e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class MissingIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final MissingIfTypeComputer INSTANCE = new MissingIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createMissableType(inputType);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
index 575ec52..e896ae7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NonTaggedGetItemResultType extends AbstractResultTypeComputer {
 
@@ -37,15 +38,17 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0) {
             if (type.getTypeTag() != ATypeTag.MULTISET && type.getTypeTag() != ATypeTag.ARRAY) {
-                throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.ARRAY);
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                        ATypeTag.ARRAY);
             }
         } else {
             if (!ATypeHierarchy.isCompatible(type.getTypeTag(), ATypeTag.INTEGER)) {
-                throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java
new file mode 100644
index 0000000..590d8f6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class NullIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final NullIfTypeComputer INSTANCE = new NullIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createNullableType(inputType);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
index 034309b..b311b6e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
@@ -36,8 +36,8 @@
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expr;
-        String funcName = functionCallExpression.getFunctionIdentifier().getName();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        String funcName = funcExpr.getFunctionIdentifier().getName();
         IAType t1 = strippedInputTypes[0];
         IAType t2 = strippedInputTypes[1];
         ATypeTag tag1 = t1.getTypeTag();
@@ -59,7 +59,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case FLOAT:
@@ -78,7 +78,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case BIGINT:
@@ -99,7 +99,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case INTEGER:
@@ -122,7 +122,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case SMALLINT:
@@ -147,7 +147,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case TINYINT:
@@ -174,7 +174,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case ANY:
@@ -189,7 +189,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DATE:
@@ -206,7 +206,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case TIME:
@@ -223,7 +223,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DATETIME:
@@ -237,7 +237,7 @@
                         type = BuiltinType.ADATETIME;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DURATION:
@@ -255,7 +255,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case YEARMONTHDURATION:
@@ -276,7 +276,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DAYTIMEDURATION:
@@ -297,11 +297,11 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             default:
-                throw new IncompatibleTypeException(funcName, tag1, tag2);
+                throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
         }
         return type;
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
index ec9c59f..32a6487 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericAggTypeComputer extends AbstractResultTypeComputer {
     public static final NumericAggTypeComputer INSTANCE = new NumericAggTypeComputer();
@@ -34,7 +35,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case DOUBLE:
@@ -46,7 +48,7 @@
             case ANY:
                 break;
             default:
-                throw new UnsupportedTypeException(funcName, tag);
+                throw new UnsupportedTypeException(sourceLoc, funcName, tag);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
new file mode 100644
index 0000000..5e2ebae
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.IncompatibleTypeException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+/**
+ * Returns double if both operands are integers
+ */
+public class NumericDivideTypeComputer extends AbstractResultTypeComputer {
+    public static final NumericDivideTypeComputer INSTANCE = new NumericDivideTypeComputer();
+
+    private NumericDivideTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        String funcName = funcExpr.getFunctionIdentifier().getName();
+        IAType t1 = strippedInputTypes[0];
+        IAType t2 = strippedInputTypes[1];
+        ATypeTag tag1 = t1.getTypeTag();
+        ATypeTag tag2 = t2.getTypeTag();
+
+        IAType type;
+        switch (tag1) {
+            case DOUBLE:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case FLOAT:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                        type = BuiltinType.AFLOAT;
+                        break;
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case BIGINT:
+            case INTEGER:
+            case SMALLINT:
+            case TINYINT:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case FLOAT:
+                        type = BuiltinType.AFLOAT;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case ANY:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                    case ANY:
+                    case DOUBLE:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DATE:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case TIME:
+                switch (tag2) {
+                    case TIME:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DATETIME:
+                switch (tag2) {
+                    case DATETIME:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case YEARMONTHDURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case YEARMONTHDURATION:
+                        type = BuiltinType.AYEARMONTHDURATION;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DAYTIMEDURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case DAYTIMEDURATION:
+                        type = BuiltinType.ADAYTIMEDURATION;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            default:
+                throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+        }
+        return type;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
index 1e9678a..54e1bcd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericDoubleOutputFunctionTypeComputer extends AbstractResultTypeComputer {
 
@@ -36,7 +37,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -47,7 +49,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
index 2081b9b..8d84ce0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericInt8OutputFunctionTypeComputer extends AbstractResultTypeComputer {
 
@@ -35,7 +36,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -46,7 +48,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
index dbec30e..b537ffc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericRound2TypeComputer extends AbstractResultTypeComputer {
 
@@ -39,7 +40,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (argIndex == 0) {
             switch (tag) {
@@ -51,8 +53,8 @@
                 case DOUBLE:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
             }
         }
         if (argIndex == 1) {
@@ -63,8 +65,8 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
index ea5c2bb..ef9791f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericUnaryFunctionTypeComputer extends AbstractResultTypeComputer {
     public static final NumericUnaryFunctionTypeComputer INSTANCE = new NumericUnaryFunctionTypeComputer();
@@ -37,7 +38,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -49,7 +51,7 @@
             case ANY:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
index a1fb5b0..5c59af6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
@@ -63,7 +63,8 @@
         IAType type0 = (IAType) env.getType(funcExpr.getArguments().get(0).getValue());
         ARecordType inputRecordType = TypeComputeUtils.extractRecordType(type0);
         if (inputRecordType == null) {
-            throw new TypeMismatchException(funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(funcExpr.getSourceLocation(), funcName, 0, type0.getTypeTag(),
+                    ATypeTag.OBJECT);
         }
 
         ILogicalExpression arg1 = funcExpr.getArguments().get(1).getValue();
@@ -109,8 +110,8 @@
                         if (fn[j].equals(FIELD_NAME_NAME)) {
                             ILogicalExpression fieldNameExpr = recConsExpr.getArguments().get(j).getValue();
                             if (ConstantExpressionUtil.getStringConstant(fieldNameExpr) == null) {
-                                throw new InvalidExpressionException(funcName, 1, fieldNameExpr,
-                                        LogicalExpressionTag.CONSTANT);
+                                throw new InvalidExpressionException(funcExpr.getSourceLocation(), funcName, 1,
+                                        fieldNameExpr, LogicalExpressionTag.CONSTANT);
                             }
                             // Get the actual "field-name" string
                             fieldName = ConstantExpressionUtil.getStringArgument(recConsExpr, j + 1);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
index db4d1c4..98ec9e7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class RecordMergeTypeComputer implements IResultTypeComputer {
     public static final RecordMergeTypeComputer INSTANCE = new RecordMergeTypeComputer();
@@ -56,12 +57,12 @@
         boolean unknownable = TypeHelper.canBeUnknown(t0) || TypeHelper.canBeUnknown(t1);
         ARecordType recType0 = TypeComputeUtils.extractRecordType(t0);
         if (recType0 == null) {
-            throw new TypeMismatchException(funcName, 0, t0.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(f.getSourceLocation(), funcName, 0, t0.getTypeTag(), ATypeTag.OBJECT);
         }
 
         ARecordType recType1 = TypeComputeUtils.extractRecordType(t1);
         if (recType1 == null) {
-            throw new TypeMismatchException(funcName, 1, t1.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(f.getSourceLocation(), funcName, 1, t1.getTypeTag(), ATypeTag.OBJECT);
         }
 
         List<String> resultFieldNames = new ArrayList<>();
@@ -90,11 +91,13 @@
             if (pos >= 0) {
                 IAType resultFieldType = resultFieldTypes.get(pos);
                 if (resultFieldType.getTypeTag() != fieldTypes[i].getTypeTag()) {
-                    throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldNames[i]);
+                    throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, f.getSourceLocation(),
+                            fieldNames[i]);
                 }
                 // Assuming fieldTypes[i].getTypeTag() = resultFieldType.getTypeTag()
                 if (fieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
-                    resultFieldTypes.set(pos, mergedNestedType(fieldNames[i], fieldTypes[i], resultFieldType));
+                    resultFieldTypes.set(pos,
+                            mergedNestedType(fieldNames[i], fieldTypes[i], resultFieldType, f.getSourceLocation()));
                 }
             } else {
                 additionalFieldNames.add(fieldNames[i]);
@@ -116,9 +119,10 @@
         return resultType;
     }
 
-    private IAType mergedNestedType(String fieldName, IAType fieldType1, IAType fieldType0) throws AlgebricksException {
+    private IAType mergedNestedType(String fieldName, IAType fieldType1, IAType fieldType0, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (fieldType1.getTypeTag() != ATypeTag.OBJECT || fieldType0.getTypeTag() != ATypeTag.OBJECT) {
-            throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldName);
+            throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, sourceLoc, fieldName);
         }
 
         ARecordType resultType = (ARecordType) fieldType0;
@@ -131,8 +135,8 @@
                 // If a sub-record do merge, else ignore and let the values decide what to do
                 if (fieldType1Copy.getFieldTypes()[i].getTypeTag() == ATypeTag.OBJECT) {
                     IAType[] oldTypes = resultType.getFieldTypes();
-                    oldTypes[pos] =
-                            mergedNestedType(fname, fieldType1Copy.getFieldTypes()[i], resultType.getFieldTypes()[pos]);
+                    oldTypes[pos] = mergedNestedType(fname, fieldType1Copy.getFieldTypes()[i],
+                            resultType.getFieldTypes()[pos], sourceLoc);
                     resultType = new ARecordType(resultType.getTypeName(), resultType.getFieldNames(), oldTypes,
                             resultType.isOpen());
                 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java
index ceba3fb..ae51c61 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class RecordPairsTypeComputer extends AbstractResultTypeComputer {
 
@@ -34,10 +35,11 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag typeTag = type.getTypeTag();
         if (typeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, typeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, typeTag, ATypeTag.OBJECT);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
index e0b10d1..4e9a563 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Cases to support:
@@ -65,10 +66,10 @@
     }
 
     private void getPathFromConstantExpression(String funcName, ILogicalExpression expression, Set<String> fieldNameSet,
-            List<List<String>> pathList) throws AlgebricksException {
+            List<List<String>> pathList, SourceLocation sourceLoc) throws AlgebricksException {
         ConstantExpression ce = (ConstantExpression) expression;
         if (!(ce.getValue() instanceof AsterixConstantValue)) {
-            throw new InvalidExpressionException(funcName, 1, ce, LogicalExpressionTag.CONSTANT);
+            throw new InvalidExpressionException(sourceLoc, funcName, 1, ce, LogicalExpressionTag.CONSTANT);
         }
         IAObject item = ((AsterixConstantValue) ce.getValue()).getObject();
         ATypeTag type = item.getType().getTypeTag();
@@ -89,11 +90,11 @@
                 pathList.add(path);
                 break;
             default:
-                throw new UnsupportedTypeException(funcName, type);
+                throw new UnsupportedTypeException(sourceLoc, funcName, type);
         }
     }
 
-    private List<String> getListFromExpression(String funcName, ILogicalExpression expression)
+    private List<String> getListFromExpression(String funcName, ILogicalExpression expression, SourceLocation sourceLoc)
             throws AlgebricksException {
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) expression;
         List<Mutable<ILogicalExpression>> args = funcExp.getArguments();
@@ -104,14 +105,14 @@
             // Input list has only one level of nesting (list of list or list of strings)
             ConstantExpression ce = (ConstantExpression) arg.getValue();
             if (!(ce.getValue() instanceof AsterixConstantValue)) {
-                throw new InvalidExpressionException(funcName, 1, ce, LogicalExpressionTag.CONSTANT);
+                throw new InvalidExpressionException(sourceLoc, funcName, 1, ce, LogicalExpressionTag.CONSTANT);
             }
             IAObject item = ((AsterixConstantValue) ce.getValue()).getObject();
             ATypeTag type = item.getType().getTypeTag();
             if (type == ATypeTag.STRING) {
                 list.add(((AString) item).getStringValue());
             } else {
-                throw new UnsupportedTypeException(funcName, type);
+                throw new UnsupportedTypeException(sourceLoc, funcName, type);
             }
         }
 
@@ -119,8 +120,8 @@
     }
 
     private void getPathFromFunctionExpression(String funcName, ILogicalExpression expression, Set<String> fieldNameSet,
-            List<List<String>> pathList) throws AlgebricksException {
-        List<String> path = getListFromExpression(funcName, expression);
+            List<List<String>> pathList, SourceLocation sourceLoc) throws AlgebricksException {
+        List<String> path = getListFromExpression(funcName, expression, sourceLoc);
         // Add the path head to remove set
         fieldNameSet.add(path.get(0));
         pathList.add(path);
@@ -136,14 +137,14 @@
             ILogicalExpression le = arg.getValue();
             switch (le.getExpressionTag()) {
                 case CONSTANT:
-                    getPathFromConstantExpression(funcName, le, fieldNameSet, pathList);
+                    getPathFromConstantExpression(funcName, le, fieldNameSet, pathList, funcExp.getSourceLocation());
                     break;
                 case FUNCTION_CALL:
-                    getPathFromFunctionExpression(funcName, le, fieldNameSet, pathList);
+                    getPathFromFunctionExpression(funcName, le, fieldNameSet, pathList, funcExp.getSourceLocation());
                     break;
                 default:
-                    throw new InvalidExpressionException(funcName, 1, le, LogicalExpressionTag.CONSTANT,
-                            LogicalExpressionTag.FUNCTION_CALL);
+                    throw new InvalidExpressionException(funcExp.getSourceLocation(), funcName, 1, le,
+                            LogicalExpressionTag.CONSTANT, LogicalExpressionTag.FUNCTION_CALL);
             }
         }
     }
@@ -159,7 +160,7 @@
         Set<String> fieldNameSet = new HashSet<>();
         Deque<String> fieldPathStack = new ArrayDeque<>();
 
-        ARecordType inputRecordType = getRecordTypeFromType(funcName, type0);
+        ARecordType inputRecordType = getRecordTypeFromType(funcName, type0, funcExpr.getSourceLocation());
         if (inputRecordType == null) {
             return BuiltinType.ANY;
         }
@@ -168,7 +169,8 @@
         IAType inputListType = (IAType) env.getType(arg1);
         AOrderedListType inputOrderedListType = TypeComputeUtils.extractOrderedListType(inputListType);
         if (inputOrderedListType == null) {
-            throw new TypeMismatchException(funcName, 1, inputListType.getTypeTag(), ATypeTag.ARRAY);
+            throw new TypeMismatchException(funcExpr.getSourceLocation(), funcName, 1, inputListType.getTypeTag(),
+                    ATypeTag.ARRAY);
         }
 
         ATypeTag tt = inputOrderedListType.getItemType().getTypeTag();
@@ -319,7 +321,8 @@
                 destFieldTypes.toArray(new IAType[n]), isOpen);
     }
 
-    private static ARecordType getRecordTypeFromType(String funcName, IAType type0) throws AlgebricksException {
+    private static ARecordType getRecordTypeFromType(String funcName, IAType type0, SourceLocation sourceLoc)
+            throws AlgebricksException {
         switch (type0.getTypeTag()) {
             case OBJECT:
                 return (ARecordType) type0;
@@ -334,7 +337,7 @@
                 }
                 // Falls through for other cases.
             default:
-                throw new TypeMismatchException(funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
+                throw new TypeMismatchException(sourceLoc, funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index fae3d90..07c79ad 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ScalarVersionOfAggregateResultType extends AbstractResultTypeComputer {
 
@@ -36,10 +37,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.ANY && tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
index b93f215..c05b085 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
@@ -24,12 +24,14 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SleepTypeComputer extends AbstractResultTypeComputer {
     public static final SleepTypeComputer INSTANCE = new SleepTypeComputer();
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (argIndex == 1) {
             switch (type.getTypeTag()) {
                 case TINYINT:
@@ -38,7 +40,7 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, type.getTypeTag(), ATypeTag.TINYINT,
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, type.getTypeTag(), ATypeTag.TINYINT,
                             ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
index e29b6ec..466ba4d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class StringIntToStringTypeComputer extends AbstractResultTypeComputer {
     public static final StringIntToStringTypeComputer INSTANCE = new StringIntToStringTypeComputer(1);
@@ -38,11 +39,12 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (argIndex < stringArgCount) {
             if (tag != ATypeTag.STRING) {
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.STRING);
             }
         } else {
             switch (tag) {
@@ -52,8 +54,8 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
index 0ab49d0..5655454 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
@@ -55,7 +55,8 @@
             case UNION: {
                 AUnionType ut = (AUnionType) t;
                 if (!ut.isUnknownableType()) {
-                    throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                    throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag,
+                            ATypeTag.MULTISET, ATypeTag.ARRAY);
                 }
                 IAType t2 = ut.getActualType();
                 ATypeTag tag2 = t2.getTypeTag();
@@ -63,12 +64,14 @@
                     AbstractCollectionType act = (AbstractCollectionType) t2;
                     return act.getItemType();
                 }
-                throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag, ATypeTag.MULTISET,
+                        ATypeTag.ARRAY);
             }
             case ANY:
                 return BuiltinType.ANY;
             default:
-                throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag, ATypeTag.MULTISET,
+                        ATypeTag.ARRAY);
         }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
index 74a9530..a1d46f0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
@@ -25,15 +25,17 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SubstringTypeComputer extends AbstractResultTypeComputer {
     public static final SubstringTypeComputer INSTANCE = new SubstringTypeComputer();
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (argIndex == 0 && tag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.STRING);
         }
         if (argIndex > 0 && argIndex <= 2) {
             switch (tag) {
@@ -43,8 +45,8 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
index fa2bf5b..59bfddf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
@@ -47,7 +47,8 @@
 
         int argNumber = fce.getArguments().size();
         if (argNumber < 3) {
-            throw new CompilationException(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, funcName, argNumber);
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, fce.getSourceLocation(),
+                    funcName, argNumber);
         }
         int argSize = fce.getArguments().size();
         List<IAType> types = new ArrayList<>();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
index d6825e3..806235c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
+import java.util.List;
+
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -30,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeComputeUtils {
 
@@ -40,13 +43,13 @@
     private static final byte NULL = 16;
 
     @FunctionalInterface
-    public static interface ArgTypeChecker {
-        public void checkArgTypes(int argIndex, IAType argType) throws AlgebricksException;
+    public interface ArgTypeChecker {
+        void checkArgTypes(int argIndex, IAType argType, SourceLocation argSrcLoc) throws AlgebricksException;
     }
 
     @FunctionalInterface
-    public static interface ResultTypeGenerator {
-        public IAType getResultType(ILogicalExpression expr, IAType... knownInputTypes) throws AlgebricksException;
+    public interface ResultTypeGenerator {
+        IAType getResultType(ILogicalExpression expr, IAType... knownInputTypes) throws AlgebricksException;
     }
 
     private TypeComputeUtils() {
@@ -73,9 +76,10 @@
             throws AlgebricksException {
         AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
 
-        IAType[] inputTypes = new IAType[fce.getArguments().size()];
+        List<Mutable<ILogicalExpression>> arguments = fce.getArguments();
+        IAType[] inputTypes = new IAType[arguments.size()];
         int index = 0;
-        for (Mutable<ILogicalExpression> argRef : fce.getArguments()) {
+        for (Mutable<ILogicalExpression> argRef : arguments) {
             ILogicalExpression arg = argRef.getValue();
             inputTypes[index++] = (IAType) env.getType(arg);
         }
@@ -89,7 +93,7 @@
                     || argTypeTag == ATypeTag.MISSING) {
                 continue;
             }
-            checker.checkArgTypes(argIndex, knownInputTypes[argIndex]);
+            checker.checkArgTypes(argIndex, knownInputTypes[argIndex], fce.getSourceLocation());
         }
 
         // Computes the result type.
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
index d52ac88..911e4d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryBinaryInt64TypeComputer extends AbstractResultTypeComputer {
     public static final UnaryBinaryInt64TypeComputer INSTANCE = new UnaryBinaryInt64TypeComputer();
@@ -34,10 +35,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.BINARY) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.BINARY);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.BINARY);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
index f815f33..3066a22 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryMinusTypeComputer extends AbstractResultTypeComputer {
 
@@ -33,7 +34,8 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -44,7 +46,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
index e744ef1..5535890 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryStringInt64TypeComputer extends AbstractResultTypeComputer {
 
@@ -34,10 +35,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index baaed59..26cbf1f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -28,7 +28,10 @@
 
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -226,6 +229,39 @@
     }
 
     /**
+     *
+     * @param subFieldName
+     *            The full pathname of the field
+     * @return The nullability of the field
+     * @throws AlgebricksException
+     */
+    public boolean isSubFieldNullable(List<String> subFieldName) throws AlgebricksException {
+        IAType subRecordType = getFieldType(subFieldName.get(0));
+        for (int i = 1; i < subFieldName.size(); i++) {
+            if (subRecordType == null) {
+                // open field is nullable
+                return true;
+            }
+            if (subRecordType.getTypeTag().equals(ATypeTag.UNION)) {
+                if (NonTaggedFormatUtil.isOptional(subRecordType)) {
+                    return true;
+                }
+                subRecordType = ((AUnionType) subRecordType).getActualType();
+                if (subRecordType.getTypeTag() != ATypeTag.OBJECT) {
+                    throw new AsterixException(
+                            "Field accessor is not defined for values of type " + subRecordType.getTypeTag());
+                }
+            }
+            if (!(subRecordType instanceof ARecordType)) {
+                throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        "Illegal field type " + subRecordType.getTypeTag() + " when checking field nullability");
+            }
+            subRecordType = ((ARecordType) subRecordType).getFieldType(subFieldName.get(i));
+        }
+        return subRecordType == null || NonTaggedFormatUtil.isOptional(subRecordType);
+    }
+
+    /**
      * Returns the field type of the field name if it exists, otherwise null.
      *
      * @param fieldName
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
index 825ed70..a79588d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
@@ -66,7 +66,8 @@
     DAYTIMEDURATION(37),
     UUID(38),
     SHORTWITHOUTTYPEINFO(40),
-    NULL(41);
+    NULL(41),
+    GEOMETRY(42);
 
     /*
      * Serialized Tags begin
@@ -100,6 +101,7 @@
     public static final byte SERIALIZED_FLOAT_TYPE_TAG = FLOAT.serialize();
     public static final byte SERIALIZED_BINARY_TYPE_TAG = BINARY.serialize();
     public static final byte SERIALIZED_UUID_TYPE_TAG = UUID.serialize();
+    public static final byte SERIALIZED_GEOMETRY_TYPE_TAG = GEOMETRY.serialize();
 
     /*
      * Serialized Tags end
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index 3037e7c..a36e0e4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -682,6 +682,33 @@
         }
     };
 
+    public final static BuiltinType AGEOMETRY = new LowerCaseConstructorType() {
+
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public String getDisplayName() {
+            return "AGEOMETRY";
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.GEOMETRY;
+        }
+
+        @Override
+        public String getTypeName() {
+            return "geometry";
+        }
+
+        @Override
+        public ObjectNode toJSON() {
+            ObjectNode type = new ObjectMapper().createObjectNode();
+            type.put("type", "AGEOMETRY");
+            return type;
+        }
+    };
+
     public final static BuiltinType ACIRCLE = new LowerCaseConstructorType() {
 
         private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
index 05f265c..f2d1f1a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
@@ -18,24 +18,25 @@
  */
 package org.apache.asterix.om.types;
 
-import java.util.HashMap;
-import java.util.Map;
+import java.lang.reflect.Array;
 
 public class EnumDeserializer<E extends Enum<E> & IEnumSerializer> {
 
     public static final EnumDeserializer<ATypeTag> ATYPETAGDESERIALIZER =
             new EnumDeserializer<ATypeTag>(ATypeTag.class);
 
-    private Map<Byte, E> enumvalMap = new HashMap<Byte, E>();
+    private final E[] enumvalMap;
 
+    @SuppressWarnings("unchecked")
     private EnumDeserializer(Class<E> enumClass) {
+        enumvalMap = (E[]) Array.newInstance(enumClass, Byte.MAX_VALUE);
         for (E constant : enumClass.getEnumConstants()) {
-            enumvalMap.put(constant.serialize(), constant);
+            enumvalMap[constant.serialize()] = constant;
         }
     }
 
     public E deserialize(byte value) {
-        return enumvalMap.get(value);
+        return enumvalMap[value];
     }
 
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
index cfc1b55..254dbee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
@@ -88,6 +88,8 @@
                 return AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE;
             case ARRAY:
                 return AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+            case GEOMETRY:
+                return BuiltinType.AGEOMETRY;
             default:
                 // TODO(tillw) should be an internal error
                 throw new HyracksDataException("Typetag " + typeTag + " is not a built-in type");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
index 8558538..7da0263 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
@@ -68,6 +69,7 @@
             case ARRAY:
             case MULTISET:
             case POLYGON:
+            case GEOMETRY:
             case ANY:
                 return false;
             default:
@@ -193,6 +195,14 @@
                 } else {
                     return AUnorderedListSerializerDeserializer.getUnorderedListLength(serNonTaggedAObject, offset) - 1;
                 }
+            case GEOMETRY:
+                // Since Geometry is variable size, we store its size at the first 32 bits for efficiency
+                // @see: STGeomFromTextDescriptor#createEvaluatorFactory, AGeometrySerializerDeserializer#serialize
+                if (tagged) {
+                    return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset + 1) + 4;
+                } else {
+                    return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset) + 4;
+                }
             default:
                 throw new NotImplementedException(
                         "No getLength implemented for a value of this type " + typeTag + " .");
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
index a94f073..2ed2ac9 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -57,7 +57,7 @@
         }
     }
 
-    public synchronized void isFull(boolean full) {
+    public synchronized void setFull(boolean full) {
         this.full.set(full);
         this.notify();
     }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
index 7a9d75f..6c8e372 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -167,7 +167,7 @@
 
     private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
         if (!currentTxnLogBuffer.hasSpace(logRecord)) {
-            currentTxnLogBuffer.isFull(true);
+            currentTxnLogBuffer.setFull(true);
             if (logRecord.getLogSize() > logPageSize) {
                 getAndInitNewLargePage(logRecord.getLogSize());
             } else {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 84922cd..d4d601c 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -22,16 +22,20 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.file.Path;
 import java.util.Collection;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
 import org.apache.hyracks.storage.common.LocalResource;
 
 /**
@@ -51,13 +55,27 @@
                 appCtx.getIndexCheckpointManagerProvider();
         PersistentLocalResourceRepository resRepo =
                 (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        final IIOManager ioManager = appCtx.getIoManager();
         final Collection<LocalResource> partitionResources = resRepo.getPartitionResources(partition).values();
         final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
         for (LocalResource ls : partitionResources) {
-            final IIndexCheckpointManager indexCheckpointManager =
-                    indexCheckpointManagerProvider.get(DatasetResourceReference.of(ls));
+            DatasetResourceReference ref = DatasetResourceReference.of(ls);
+            final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(ref);
             indexCheckpointManager.delete();
-            indexCheckpointManager.init(currentLSN);
+            // Get most recent timestamp of existing files to avoid deletion
+            Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
+            String[] files = indexPath.toFile().list(AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER);
+            if (files == null) {
+                throw HyracksDataException
+                        .create(new IOException(indexPath + " is not a directory or an IO Error occurred"));
+            }
+            String mostRecentTimestamp = null;
+            for (String file : files) {
+                String nextTimeStamp = AbstractLSMIndexFileManager.getComponentEndTime(file);
+                mostRecentTimestamp = mostRecentTimestamp == null || nextTimeStamp.compareTo(mostRecentTimestamp) > 0
+                        ? nextTimeStamp : mostRecentTimestamp;
+            }
+            indexCheckpointManager.init(mostRecentTimestamp, currentLSN);
         }
         ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
     }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
index 57474ef..a4f9b43 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.replication.api.IReplicaTask;
 import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
 
@@ -43,17 +44,21 @@
 public class MarkComponentValidTask implements IReplicaTask {
 
     private final long masterLsn;
+    private final long lastComponentId;
     private final String file;
 
-    public MarkComponentValidTask(String file, long masterLsn) {
+    public MarkComponentValidTask(String file, long masterLsn, long lastComponentId) {
         this.file = file;
+        this.lastComponentId = lastComponentId;
         this.masterLsn = masterLsn;
     }
 
     @Override
     public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
         try {
-            if (masterLsn > 0) {
+            if (masterLsn == IndexSynchronizer.BULKLOAD_LSN) {
+                updateBulkLoadedLastComponentTimestamp(appCtx);
+            } else if (masterLsn != IndexSynchronizer.MERGE_LSN) {
                 ensureComponentLsnFlushed(appCtx);
             }
             // delete mask
@@ -65,6 +70,15 @@
         }
     }
 
+    private void updateBulkLoadedLastComponentTimestamp(INcApplicationContext appCtx) throws HyracksDataException {
+        final ResourceReference indexRef = ResourceReference.of(file);
+        final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+        final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+        final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
+        indexCheckpointManager.advanceValidComponentTimestamp(componentEndTime);
+
+    }
+
     private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
             throws HyracksDataException, InterruptedException {
         final ResourceReference indexRef = ResourceReference.of(file);
@@ -82,7 +96,7 @@
                 replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
             }
             final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
-            indexCheckpointManager.replicated(componentEndTime, masterLsn);
+            indexCheckpointManager.replicated(componentEndTime, masterLsn, lastComponentId);
         }
     }
 
@@ -97,6 +111,7 @@
             final DataOutputStream dos = new DataOutputStream(out);
             dos.writeUTF(file);
             dos.writeLong(masterLsn);
+            dos.writeLong(lastComponentId);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
@@ -105,6 +120,7 @@
     public static MarkComponentValidTask create(DataInput input) throws IOException {
         final String indexFile = input.readUTF();
         final long lsn = input.readLong();
-        return new MarkComponentValidTask(indexFile, lsn);
+        final long lastComponentId = input.readLong();
+        return new MarkComponentValidTask(indexFile, lsn, lastComponentId);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index ca0fcca..20663d1 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -30,12 +30,12 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.replication.management.NetworkingUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -98,7 +98,7 @@
         final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
         final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
         indexCheckpointManager.delete();
-        indexCheckpointManager.init(currentLSN);
+        indexCheckpointManager.init(null, currentLSN);
         LOGGER.info(() -> "Checkpoint index: " + indexRef);
     }
 
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
index 95ae690..30a5595 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -24,7 +24,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.replication.messaging.ComponentMaskTask;
@@ -37,12 +37,15 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 public class IndexSynchronizer {
 
     private static final Logger LOGGER = LogManager.getLogger();
+    public static final long MERGE_LSN = -1;
+    public static final long BULKLOAD_LSN = -2;
     private final IReplicationJob job;
     private final INcApplicationContext appCtx;
 
@@ -91,7 +94,8 @@
         final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
         job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
         // send mark component valid
-        MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn());
+        MarkComponentValidTask markValidTask =
+                new MarkComponentValidTask(indexFile, getReplicatedComponentLsn(), getReplicatedComponentId());
         ReplicationProtocol.sendTo(replica, markValidTask);
         ReplicationProtocol.waitForAck(replica);
         LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
@@ -118,12 +122,28 @@
 
     private long getReplicatedComponentLsn() throws HyracksDataException {
         final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+        if (indexReplJob.getLSMOpType() == LSMOperationType.MERGE) {
+            return MERGE_LSN;
+        } else if (indexReplJob.getLSMOpType() == LSMOperationType.LOAD) {
+            return BULKLOAD_LSN;
+        }
+
         if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
-            return AbstractLSMIOOperationCallback.INVALID;
+            return LSMIOOperationCallback.INVALID_LSN;
         }
         final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
         final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
-        return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+        return ((LSMIOOperationCallback) lsmIndex.getIOOperationCallback())
                 .getComponentLSN(ctx.getComponentsToBeReplicated());
     }
+
+    private long getReplicatedComponentId() throws HyracksDataException {
+        final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+        if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+            return -1L;
+        }
+        final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
+        LSMComponentId id = (LSMComponentId) ctx.getComponentsToBeReplicated().get(0).getId();
+        return id.getMinId();
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
index 34f961c..6566a04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
@@ -38,7 +38,7 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new FirstElementEvalFactory(args, false);
+        return new FirstElementEvalFactory(args, false, sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
index bea8d5f..86785e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
@@ -19,12 +19,14 @@
 package org.apache.asterix.runtime.aggregates.collections;
 
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -35,15 +37,17 @@
     private static final long serialVersionUID = 1L;
     private final IScalarEvaluatorFactory[] args;
     private final boolean isLocal;
+    private final SourceLocation sourceLoc;
 
-    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal) {
+    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
         this.args = args;
         this.isLocal = isLocal;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-        return new IAggregateEvaluator() {
+        return new AbstractAggregateFunction(sourceLoc) {
 
             private boolean first = true;
             // Needs to copy the bytes from inputVal to outputVal because the byte space of inputVal could be re-used
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index 4e35950..6cc119e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -58,6 +58,6 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new ListifyAggregateFunctionEvalFactory(args, oltype);
+        return new ListifyAggregateFunctionEvalFactory(args, oltype, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
index 2bb9eb3..e2e2859 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
@@ -22,12 +22,14 @@
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -38,15 +40,18 @@
     private static final long serialVersionUID = 1L;
     private IScalarEvaluatorFactory[] args;
     private final AOrderedListType orderedlistType;
+    private final SourceLocation sourceLoc;
 
-    public ListifyAggregateFunctionEvalFactory(IScalarEvaluatorFactory[] args, AOrderedListType type) {
+    public ListifyAggregateFunctionEvalFactory(IScalarEvaluatorFactory[] args, AOrderedListType type,
+            SourceLocation sourceLoc) {
         this.args = args;
         this.orderedlistType = type;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-        return new IAggregateEvaluator() {
+        return new AbstractAggregateFunction(sourceLoc) {
 
             private IPointable inputVal = new VoidPointable();
             private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
index 5c73390..baf89e0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
@@ -37,7 +37,7 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new FirstElementEvalFactory(args, true);
+        return new FirstElementEvalFactory(args, true, sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index e929f0f..c8ebff6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -56,7 +56,7 @@
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 // Use ScanCollection to iterate over list items.
                 ScanCollectionUnnestingFunctionFactory scanCollectionFactory =
-                        new ScanCollectionUnnestingFunctionFactory(args[0]);
+                        new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
                 return new GenericScalarAggregateFunction(aggFuncFactory.createAggregateEvaluator(ctx),
                         scanCollectionFactory, ctx);
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java
new file mode 100644
index 0000000..3054323
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractSerializableAggregateFunction implements ISerializedAggregateEvaluator {
+    protected final SourceLocation sourceLoc;
+
+    protected AbstractSerializableAggregateFunction(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
index 9d4712b7..563ff00 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
@@ -49,16 +49,16 @@
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSerializableAvgAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableAvgAggregateFunction extends AbstractSerializableAggregateFunction {
     private static final int SUM_FIELD_ID = 0;
     private static final int COUNT_FIELD_ID = 1;
 
@@ -90,8 +90,9 @@
     private ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -136,7 +137,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.AVG, bytes[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset], aggType.serialize());
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -173,7 +174,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, bytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset]);
         }
         BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
         BufferSerDeUtil.writeLong(count, state, start + COUNT_OFFSET);
@@ -252,7 +253,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, serBytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, serBytes[offset]);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
index a2a2d1f..c37a524 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
@@ -31,9 +31,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -41,7 +41,7 @@
 /**
  * count(NULL) returns NULL.
  */
-public abstract class AbstractSerializableCountAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableCountAggregateFunction extends AbstractSerializableAggregateFunction {
     private static final int MET_NULL_OFFSET = 0;
     private static final int COUNT_OFFSET = 1;
 
@@ -55,8 +55,9 @@
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
 
-    public AbstractSerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
index 8ba3a05..a4e1090 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
@@ -44,15 +44,15 @@
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSerializableSumAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableSumAggregateFunction extends AbstractSerializableAggregateFunction {
     protected static final int AGG_TYPE_OFFSET = 0;
     private static final int SUM_OFFSET = 1;
 
@@ -67,8 +67,9 @@
     @SuppressWarnings("rawtypes")
     public ISerializerDeserializer serde;
 
-    public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -100,7 +101,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.SUM, bytes[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset], aggType.serialize());
         }
 
         if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -147,7 +148,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, bytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset]);
         }
         state[start + AGG_TYPE_OFFSET] = aggType.serialize();
         BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
@@ -206,7 +207,7 @@
                     break;
                 }
                 default:
-                    throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
index ab4aa63..6601243 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableAvgAggregateFunction(args, ctx);
+                return new SerializableAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
index 4c32fe0..611538f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
index ec86cf8..506a9f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
@@ -56,7 +56,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableCountAggregateFunction(args, ctx);
+                return new SerializableCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
index fb8d8d7..0815dfd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
@@ -21,13 +21,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * count(NULL) returns NULL.
  */
 public class SerializableCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
-    public SerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
index 8f40f8a..f9a8834 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableGlobalAvgAggregateFunction(args, ctx);
+                return new SerializableGlobalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
index a916355..9bb4312 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableGlobalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableGlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableGlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
index 19149d5..c67da27 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableGlobalSqlAvgAggregateFunction(args, ctx);
+                return new SerializableGlobalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
index 8ddccdc..5fe1d7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableGlobalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableGlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableGlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
index 28b3aff..676eabb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableIntermediateAvgAggregateFunction(args, ctx);
+                return new SerializableIntermediateAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
index 3d56133..d25f603 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableIntermediateAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableIntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableIntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
index 43e63cd..86bb15d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableIntermediateSqlAvgAggregateFunction(args, ctx);
+                return new SerializableIntermediateSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
index f8449a5..d3f9fea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableIntermediateSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableIntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableIntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
index 9c94ad2..39ae425 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableLocalAvgAggregateFunction(args, ctx);
+                return new SerializableLocalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
index 0e7ba07..f93bbd7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableLocalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableLocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableLocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
index d0819e0..c96eb2e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableLocalSqlAvgAggregateFunction(args, ctx);
+                return new SerializableLocalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
index 8925c31..c6195aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableLocalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableLocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableLocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
index 2cd75f1..3798b49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, true, ctx);
+                return new SerializableSqlSumAggregateFunction(args, true, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
index 3e0546c..54d24f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, true, ctx);
+                return new SerializableSumAggregateFunction(args, true, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
index af057ee..f34827a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlAvgAggregateFunction(args, ctx);
+                return new SerializableSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
index 3ef7aeb..608f2a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
@@ -23,13 +23,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
index 3beb59f..b44c114 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
@@ -56,7 +56,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlCountAggregateFunction(args, ctx);
+                return new SerializableSqlCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
index 257735f..241180c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
@@ -21,14 +21,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * count(NULL) returns NULL.
  */
 public class SerializableSqlCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
-    public SerializableSqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableSqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
index 4e8d515..d51a6fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, false, ctx);
+                return new SerializableSqlSumAggregateFunction(args, false, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
index c42c350..38033f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
@@ -30,13 +30,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SerializableSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
     private final boolean isLocalAgg;
 
     public SerializableSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context) throws HyracksDataException {
-        super(args, context);
+            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -50,7 +51,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SQL_SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
index 8d2bb6c..43eea5b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, false, ctx);
+                return new SerializableSumAggregateFunction(args, false, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
index 60e34fa..278914f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
@@ -31,13 +31,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SerializableSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
     private final boolean isLocalAgg;
 
     public SerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context) throws HyracksDataException {
-        super(args, context);
+            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -58,7 +59,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java
new file mode 100644
index 0000000..1ea28cc
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractAggregateFunction implements IAggregateEvaluator {
+    protected final SourceLocation sourceLoc;
+
+    public AbstractAggregateFunction(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
index 37cf766..1dde27b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
@@ -47,19 +47,19 @@
 import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractAvgAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractAvgAggregateFunction extends AbstractAggregateFunction {
     private static final int SUM_FIELD_ID = 0;
     private static final int COUNT_FIELD_ID = 1;
 
@@ -93,8 +93,9 @@
     private ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
         recType = new ARecordType(null, new String[] { "sum", "count" },
                 new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
@@ -134,7 +135,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.AVG, data[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset], aggType.serialize());
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -171,7 +172,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, data[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset]);
             }
         }
     }
@@ -234,7 +235,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException("intermediate/global-Avg", serBytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, "intermediate/global-Avg", serBytes[offset]);
             }
         }
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
index 6dc4759..d156574 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
@@ -27,11 +27,11 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -40,7 +40,7 @@
 /**
  * COUNT returns the number of items in the given list. Note that COUNT(NULL) is not allowed.
  */
-public abstract class AbstractCountAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractCountAggregateFunction extends AbstractAggregateFunction {
     private AMutableInt64 result = new AMutableInt64(-1);
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<AInt64> int64Serde =
@@ -51,8 +51,9 @@
 
     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
 
-    public AbstractCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
index 0ba7c71..1aa609f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
@@ -26,19 +26,19 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.om.types.hierachy.ITypeConvertComputer;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractMinMaxAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractMinMaxAggregateFunction extends AbstractAggregateFunction {
     private IPointable inputVal = new VoidPointable();
     private ArrayBackedValueStorage outputVal = new ArrayBackedValueStorage();
     private ArrayBackedValueStorage tempValForCasting = new ArrayBackedValueStorage();
@@ -50,8 +50,9 @@
     private ITypeConvertComputer tpc;
     private final boolean isMin;
 
-    public AbstractMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin)
-            throws HyracksDataException {
+    public AbstractMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
         this.isMin = isMin;
     }
@@ -87,7 +88,7 @@
             // Initialize min value.
             outputVal.assign(inputVal);
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException("min/max", typeTag.serialize(), aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, "min/max", typeTag.serialize(), aggType.serialize());
         } else {
 
             // If a system_null is encountered locally, it would be an error; otherwise if it is seen
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
index fd841e0..a8fa552 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
@@ -41,18 +41,18 @@
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSumAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractSumAggregateFunction extends AbstractAggregateFunction {
     protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
@@ -67,8 +67,9 @@
     @SuppressWarnings("rawtypes")
     protected ISerializerDeserializer serde;
 
-    public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -94,7 +95,8 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.SUM, typeTag.serialize(), aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, typeTag.serialize(),
+                    aggType.serialize());
         }
 
         if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -137,7 +139,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         }
     }
@@ -194,7 +196,7 @@
                     break;
                 }
                 default:
-                    throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
index 93023e8..46fe3e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new AvgAggregateFunction(args, ctx);
+                return new AvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
index 6408e56..3572c4f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class AvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public AvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public AvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
-        super(args, context);
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
index f981ece..9f64f19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
@@ -54,7 +54,7 @@
 
             @Override
             public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new CountAggregateFunction(args, ctx);
+                return new CountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
index 94b653d..829e8ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
@@ -21,15 +21,16 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * COUNT returns the number of items in the given list. Note that COUNT(NULL) is not allowed.
  */
 public class CountAggregateFunction extends AbstractCountAggregateFunction {
 
-    public CountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public CountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
-        super(args, context);
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
index 199e5e5..a624d97 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new GlobalAvgAggregateFunction(args, ctx);
+                return new GlobalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
index 30a2746..425d928 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class GlobalAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public GlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public GlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
index 90124d1..4e3e8fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new GlobalSqlAvgAggregateFunction(args, ctx);
+                return new GlobalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
index 893613b..fb59615 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class GlobalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public GlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public GlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
index 9faccd8..391a5cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new IntermediateAvgAggregateFunction(args, ctx);
+                return new IntermediateAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
index ad64ef3..c182fcd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class IntermediateAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public IntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public IntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
index 144614a..25ca512 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new IntermediateSqlAvgAggregateFunction(args, ctx);
+                return new IntermediateSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
index bc260e6..eee736d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class IntermediateSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public IntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public IntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
index 2598da1..2e23c03 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new LocalAvgAggregateFunction(args, ctx);
+                return new LocalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
index 685af83..0eab063 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class LocalAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public LocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public LocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
index b2fc3b2..ea9b2ae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, false, true);
+                return new MinMaxAggregateFunction(args, ctx, false, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
index c29149d..d4355df 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, true, true);
+                return new MinMaxAggregateFunction(args, ctx, true, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
index dc8fbc5..777d50f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new LocalSqlAvgAggregateFunction(args, ctx);
+                return new LocalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
index 8c80c42..66a75e0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class LocalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public LocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public LocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
index ab999d0..57da62b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, false, true);
+                return new SqlMinMaxAggregateFunction(args, ctx, false, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
index ee45d48..ca2f65d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, true, true);
+                return new SqlMinMaxAggregateFunction(args, ctx, true, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
index aac1927..f626d8d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, true);
+                return new SqlSumAggregateFunction(args, ctx, true, sourceLoc);
             };
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
index dd6c974..0413db4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, true);
+                return new SumAggregateFunction(args, ctx, true, sourceLoc);
             };
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
index 3afa4fb..2575440 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, false, false);
+                return new MinMaxAggregateFunction(args, ctx, false, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
index 4328618..dfff5f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, true, false);
+                return new MinMaxAggregateFunction(args, ctx, true, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
index 5d28018..3aeb58a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class MinMaxAggregateFunction extends AbstractMinMaxAggregateFunction {
     private final boolean isLocalAgg;
 
     public MinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
-            boolean isLocalAgg) throws HyracksDataException {
-        super(args, context, isMin);
+            boolean isLocalAgg, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, isMin, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -48,7 +49,7 @@
     @Override
     protected void processSystemNull() throws HyracksDataException {
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, "min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
index 0cc1b90..5f9cc46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlAvgAggregateFunction(args, ctx);
+                return new SqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
index e6c1233..8e8f60c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public SqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
index a825df4..91d31c3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
@@ -55,7 +55,7 @@
 
             @Override
             public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new SqlCountAggregateFunction(args, ctx);
+                return new SqlCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
index 7396d39..68cb37b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
@@ -21,15 +21,16 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * COUNT returns the number of non-null items in the given list. Note that COUNT(NULL) is not allowed.
  */
 public class SqlCountAggregateFunction extends AbstractCountAggregateFunction {
 
-    public SqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
index d670a66..d5b86f9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, false, false);
+                return new SqlMinMaxAggregateFunction(args, ctx, false, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
index 648dd2d..3abaa2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, true, false);
+                return new SqlMinMaxAggregateFunction(args, ctx, true, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
index 2ad0e55..f6415d6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SqlMinMaxAggregateFunction extends AbstractMinMaxAggregateFunction {
     private final boolean isLocalAgg;
 
     public SqlMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
-            boolean isLocalAgg) throws HyracksDataException {
-        super(args, context, isMin);
+            boolean isLocalAgg, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, isMin, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -42,7 +43,7 @@
     @Override
     protected void processSystemNull() throws HyracksDataException {
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, "min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
index e3b0187..a0d7708 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, false);
+                return new SqlSumAggregateFunction(args, ctx, false, sourceLoc);
             };
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
index 663ed69..8b4bf7c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
@@ -29,13 +29,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SqlSumAggregateFunction extends AbstractSumAggregateFunction {
     private final boolean isLocalAgg;
 
-    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -49,7 +50,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SQL_SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
index 34014a1..57175b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, false);
+                return new SumAggregateFunction(args, ctx, false, sourceLoc);
             };
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
index 2b3beef..abcac77 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
@@ -29,13 +29,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SumAggregateFunction extends AbstractSumAggregateFunction {
     private final boolean isLocalAgg;
 
-    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
-            throws HyracksDataException {
-        super(args, context);
+    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -55,7 +56,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
index dc2b355..fd7ccea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -58,7 +59,7 @@
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
 
-                return new IAggregateEvaluator() {
+                return new AbstractAggregateFunction(sourceLoc) {
 
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     @SuppressWarnings("rawtypes")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
index e9330d3..def848d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -56,7 +57,7 @@
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
 
-                return new IAggregateEvaluator() {
+                return new AbstractAggregateFunction(sourceLoc) {
 
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     @SuppressWarnings("rawtypes")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
index 2712a47..96ce843 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
@@ -41,7 +41,7 @@
     }
 
     @Override
-    public boolean accept(IFrameTupleReference tuple) throws Exception {
+    public boolean accept(IFrameTupleReference tuple) throws HyracksDataException {
         eval.evaluate(tuple, p);
         return boolInspector.getBooleanValue(p.getByteArray(), p.getStartOffset(), p.getLength());
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
index 98943a4..23581e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
@@ -93,7 +93,7 @@
                                 aPoint.setValue(cX, cY);
                                 pointSerde.serialize(aPoint, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
index e7a3f02..222ae76 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
@@ -88,7 +88,7 @@
                                 aDouble.setValue(radius);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
index d3069ee..60ee118 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
@@ -137,7 +137,7 @@
                                         startOffset + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                                 if (numOfPoints < 3) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
                                 listBuilder.reset(pointListType);
@@ -153,7 +153,7 @@
                                 }
                                 listBuilder.write(out, true);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_LINE_TYPE_TAG, ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
index ad9dcda..487a632 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
@@ -88,7 +88,7 @@
                                 aDouble.setValue(x);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
index 6b96ddd..9409837 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
@@ -92,7 +92,7 @@
                                 aDouble.setValue(y);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
index e9d7339..50f8f68 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
index d18864b..aa684bb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
@@ -114,7 +114,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
index c5ce56b..98d6a9d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
@@ -109,11 +109,11 @@
                                     aDateTime.setValue(endTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
index 777eb76..909b657 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
@@ -91,11 +91,11 @@
                                     aDate.setValue((int) (endTime));
                                     dateSerde.serialize(aDate, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
index 34161da..a62a8a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
@@ -93,11 +93,11 @@
                                     aDateTime.setValue(endTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
index 10a3612..4d545e8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
@@ -90,11 +90,11 @@
                                     aTime.setValue((int) (endTime));
                                     timeSerde.serialize(aTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
index e858c68..b209139 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
@@ -110,11 +110,11 @@
                                     aDateTime.setValue(startTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
index 84ed55f..901d685 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
@@ -94,11 +94,11 @@
                                     aDate.setValue((int) (startTime));
                                     dateSerde.serialize(aDate, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
index d847b52..63cd481 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
@@ -91,11 +91,11 @@
                                     aDateTime.setValue(startTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
index e59a3d2..9a1b86f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
@@ -92,11 +92,11 @@
                                     aTime.setValue((int) (startTime));
                                     timeSerde.serialize(aTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
index fcfbed3..a5101b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
@@ -112,7 +112,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
index 2a84320..6b9e573 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
@@ -115,7 +115,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
index 4709171..3b93969 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
index fe2bb39..e757e12 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
index 21dcc67..6327a36 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
@@ -132,7 +132,7 @@
                                 result.set(resultStorage);
                                 return;
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index 620c543..8194c99 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -20,19 +20,15 @@
 
 import java.io.DataOutput;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -41,85 +37,54 @@
 
 public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
 
-    protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    protected DataOutput out = resultStorage.getDataOutput();
-    protected TaggedValuePointable argLeft = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-    protected TaggedValuePointable argRight = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-    protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
-    protected IPointable outRight = VoidPointable.FACTORY.createPointable();
-    protected IScalarEvaluator evalLeft;
-    protected IScalarEvaluator evalRight;
-    private ComparisonHelper ch = new ComparisonHelper();
-
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ABoolean> serde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ANull> nullSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput out = resultStorage.getDataOutput();
+    protected final TaggedValuePointable argLeft =
+            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected final TaggedValuePointable argRight =
+            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected final IPointable outLeft = VoidPointable.FACTORY.createPointable();
+    protected final IPointable outRight = VoidPointable.FACTORY.createPointable();
+    protected final IScalarEvaluator evalLeft;
+    protected final IScalarEvaluator evalRight;
+    protected final SourceLocation sourceLoc;
+    private final ComparisonHelper ch;
+    private Number leftValue;
+    private Number rightValue;
 
     public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws HyracksDataException {
-        this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
-        this.evalRight = evalRightFactory.createScalarEvaluator(context);
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        this.evalLeft = evalLeftFactory.createScalarEvaluator(ctx);
+        this.evalRight = evalRightFactory.createScalarEvaluator(ctx);
+        this.sourceLoc = sourceLoc;
+        ch = new ComparisonHelper(sourceLoc);
+        leftValue = getValueOfConstantEval(evalLeftFactory);
+        rightValue = getValueOfConstantEval(evalRightFactory);
+    }
+
+    private Number getValueOfConstantEval(IScalarEvaluatorFactory factory) throws HyracksDataException {
+        if (factory instanceof ConstantEvalFactory) {
+            return ch.getNumberValue(((ConstantEvalFactory) factory).getValue());
+        }
+        return null;
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        resultStorage.reset();
-
         // Evaluates input args.
         evalLeft.evaluate(tuple, argLeft);
         evalRight.evaluate(tuple, argRight);
         argLeft.getValue(outLeft);
         argRight.getValue(outRight);
 
-        // checks whether we can apply >, >=, <, and <= to the given type since
-        // these operations cannot be defined for certain types.
-        if (isTotallyOrderable()) {
-            checkTotallyOrderable();
-        }
-
-        // Checks whether two types are comparable
-        if (comparabilityCheck()) {
-            // Two types can be compared
-            int r = compareResults();
-            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
-            serde.serialize(b, out);
-        } else {
-            // result:NULL - two types cannot be compared.
-            nullSerde.serialize(ANull.NULL, out);
-        }
-        result.set(resultStorage);
+        evaluateImpl(result);
     }
 
-    protected abstract boolean isTotallyOrderable();
-
-    protected abstract boolean getComparisonResult(int r);
-
-    // checks whether we can apply >, >=, <, and <= operations to the given type since
-    // these operations can not be defined for certain types.
-    protected void checkTotallyOrderable() throws HyracksDataException {
-        if (argLeft.getLength() != 0) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-            switch (typeTag) {
-                case DURATION:
-                case INTERVAL:
-                case LINE:
-                case POINT:
-                case POINT3D:
-                case POLYGON:
-                case CIRCLE:
-                case RECTANGLE:
-                    throw new UnsupportedTypeException(ComparisonHelper.COMPARISON, argLeft.getTag());
-                default:
-                    return;
-            }
-        }
-    }
+    protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
     // checks whether two types are comparable
-    protected boolean comparabilityCheck() {
+    boolean comparabilityCheck() {
         // Checks whether two types are comparable or not
         ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
         ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
@@ -128,10 +93,9 @@
         return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
     }
 
-    protected int compareResults() throws HyracksDataException {
-        int result = ch.compare(EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag()),
-                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag()), outLeft, outRight);
-        return result;
+    int compare() throws HyracksDataException {
+        ATypeTag leftTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+        ATypeTag rightTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
+        return ch.compare(leftTypeTag, rightTypeTag, outLeft, outRight, leftValue, rightValue);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
new file mode 100644
index 0000000..27ae471
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
+
+    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IScalarEvaluatorFactory evalRightFactory,
+            IHyracksTaskContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        if (comparabilityCheck() && compare() == 0) {
+            resultStorage.reset();
+            writeEqualsResult();
+            result.set(resultStorage);
+        } else {
+            result.set(argLeft);
+        }
+    }
+
+    protected abstract void writeEqualsResult() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
new file mode 100644
index 0000000..2d8c499
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractValueComparisonEvaluator extends AbstractComparisonEvaluator {
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<ABoolean> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<ANull> nullSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+
+        // checks whether we can apply >, >=, <, and <= to the given type since
+        // these operations cannot be defined for certain types.
+        if (isTotallyOrderable()) {
+            checkTotallyOrderable();
+        }
+
+        // Checks whether two types are comparable
+        if (comparabilityCheck()) {
+            // Two types can be compared
+            int r = compare();
+            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
+            serde.serialize(b, out);
+        } else {
+            // result:NULL - two types cannot be compared.
+            nullSerde.serialize(ANull.NULL, out);
+        }
+        result.set(resultStorage);
+    }
+
+    protected abstract boolean isTotallyOrderable();
+
+    protected abstract boolean getComparisonResult(int r);
+
+    // checks whether we can apply >, >=, <, and <= operations to the given type since
+    // these operations can not be defined for certain types.
+    protected void checkTotallyOrderable() throws HyracksDataException {
+        if (argLeft.getLength() != 0) {
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+            switch (typeTag) {
+                case DURATION:
+                case INTERVAL:
+                case LINE:
+                case POINT:
+                case POINT3D:
+                case POLYGON:
+                case CIRCLE:
+                case RECTANGLE:
+                    throw new UnsupportedTypeException(sourceLoc, ComparisonHelper.COMPARISON, argLeft.getTag());
+                default:
+                    return;
+            }
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
index 163bd9f..2f82173 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -37,15 +37,18 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
+
+import static org.apache.asterix.om.types.ATypeTag.TINYINT;
 
 public class ComparisonHelper implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -74,21 +77,27 @@
     private final IBinaryComparator byteArrayComparator =
             new PointableBinaryComparatorFactory(ByteArrayPointable.FACTORY).createBinaryComparator();
 
-    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+    private final SourceLocation sourceLoc;
+
+    public ComparisonHelper(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
             throws HyracksDataException {
         switch (typeTag1) {
             case TINYINT:
-                return compareInt8WithArg(typeTag2, arg1, arg2);
+                return compareInt8WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case SMALLINT:
-                return compareInt16WithArg(typeTag2, arg1, arg2);
+                return compareInt16WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case INTEGER:
-                return compareInt32WithArg(typeTag2, arg1, arg2);
+                return compareInt32WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case BIGINT:
-                return compareInt64WithArg(typeTag2, arg1, arg2);
+                return compareInt64WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case FLOAT:
-                return compareFloatWithArg(typeTag2, arg1, arg2);
+                return compareFloatWithArg(typeTag2, arg1, arg2, obj1, obj2);
             case DOUBLE:
-                return compareDoubleWithArg(typeTag2, arg1, arg2);
+                return compareDoubleWithArg(typeTag2, arg1, arg2, obj1, obj2);
             case STRING:
                 return compareStringWithArg(typeTag2, arg1, arg2);
             case BOOLEAN:
@@ -98,10 +107,16 @@
         }
     }
 
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+            throws HyracksDataException {
+        return compare(typeTag1, typeTag2, arg1, arg2, null, null);
+    }
+
     private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
             IPointable arg2) throws HyracksDataException {
         if (expectedTypeTag != actualTypeTag) {
-            throw new IncompatibleTypeException(COMPARISON, actualTypeTag.serialize(), expectedTypeTag.serialize());
+            throw new IncompatibleTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize(),
+                    expectedTypeTag.serialize());
         }
         int result;
         byte[] leftBytes = arg1.getByteArray();
@@ -160,7 +175,7 @@
                         rightLen);
                 break;
             default:
-                throw new UnsupportedTypeException(COMPARISON, actualTypeTag.serialize());
+                throw new UnsupportedTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize());
         }
         return result;
     }
@@ -171,7 +186,8 @@
             byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
             return compareByte(b0, b1);
         }
-        throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG, typeTag2.serialize());
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG,
+                typeTag2.serialize());
     }
 
     private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
@@ -179,191 +195,197 @@
             return strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
                     arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength() - 1);
         }
-        throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG, typeTag2.serialize());
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                typeTag2.serialize());
     }
 
-    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        double s = ADoubleSerializerDeserializer.getDouble(leftBytes, leftOffset);
+        double s = getOrDeserializeDouble(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareDouble(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareDouble(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareDouble(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareDouble(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareDouble(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default: {
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
                         typeTag2.serialize());
             }
         }
     }
 
-    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        float s = FloatPointable.getFloat(leftBytes, leftOffset);
+        float s = getOrDeserializeFloat(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareFloat(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareFloat(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareFloat(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareFloat(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        long s = AInt64SerializerDeserializer.getLong(leftBytes, leftOffset);
+        long s = getOrDeserializeBigInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareLong(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareLong(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareLong(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        int s = IntegerPointable.getInteger(leftBytes, leftOffset);
+        int s = getOrDeserializeInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
-            case TINYINT: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case SMALLINT: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INTEGER: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case BIGINT: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
+            case TINYINT:
+                return compareInt(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareInt(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        short s = AInt16SerializerDeserializer.getShort(leftBytes, leftOffset);
+        short s = getOrDeserializeSmallInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
-            case TINYINT: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case SMALLINT: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case INTEGER: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case BIGINT: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+            case TINYINT:
+                return compareShort(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                         typeTag2.serialize());
-            }
         }
     }
 
-    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftStart = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightStart = arg2.getStartOffset();
 
-        byte s = AInt8SerializerDeserializer.getByte(leftBytes, leftStart);
+        byte s = getOrDeserializeTinyInt(leftBytes, leftStart, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareByte(s, AInt8SerializerDeserializer.getByte(rightBytes, rightStart));
+                return compareByte(s, getOrDeserializeTinyInt(rightBytes, rightStart, obj2));
             case SMALLINT:
-                return compareShort(s, AInt16SerializerDeserializer.getShort(rightBytes, rightStart));
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightStart, obj2));
             case INTEGER:
-                return compareInt(s, AInt32SerializerDeserializer.getInt(rightBytes, rightStart));
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightStart, obj2));
             case BIGINT:
-                return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightStart));
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightStart, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightStart));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightStart, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightStart, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
+    private final byte getOrDeserializeTinyInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt8SerializerDeserializer.getByte(bytes, offset) : obj.byteValue();
+    }
+
+    private final short getOrDeserializeSmallInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt16SerializerDeserializer.getShort(bytes, offset) : obj.shortValue();
+    }
+
+    private final int getOrDeserializeInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt32SerializerDeserializer.getInt(bytes, offset) : obj.intValue();
+    }
+
+    private final long getOrDeserializeBigInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt64SerializerDeserializer.getLong(bytes, offset) : obj.longValue();
+    }
+
+    private final float getOrDeserializeFloat(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AFloatSerializerDeserializer.getFloat(bytes, offset) : obj.floatValue();
+    }
+
+    private final double getOrDeserializeDouble(byte[] bytes, int offset, Number obj) {
+        return obj == null ? ADoubleSerializerDeserializer.getDouble(bytes, offset) : obj.doubleValue();
+    }
+
     private final int compareByte(int v1, int v2) {
         if (v1 == v2) {
             return 0;
@@ -405,23 +427,41 @@
     }
 
     private final int compareFloat(float v1, float v2) {
-        if (v1 == v2) {
-            return 0;
-        } else if (v1 < v2) {
-            return -1;
-        } else {
-            return 1;
-        }
+        return Float.compare(v1, v2);
     }
 
     private final int compareDouble(double v1, double v2) {
-        if (v1 == v2) {
-            return 0;
-        } else if (v1 < v2) {
-            return -1;
-        } else {
-            return 1;
-        }
+        return Double.compare(v1, v2);
     }
 
+    /**
+     * When field value falls into the primitive type groups, we consider to cache its value instead of deserialize it
+     * every time.
+     *
+     * @param bytes
+     * @return primitive value as Number
+     */
+    public Number getNumberValue(byte[] bytes) {
+        ATypeTag aTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[0]);
+        int offset = 1;
+        if (aTypeTag == null) {
+            return null;
+        }
+        switch (aTypeTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(bytes, offset);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(bytes, offset);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(bytes, offset);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(bytes, offset);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(bytes, offset);
+            case DOUBLE:
+                return ADoubleSerializerDeserializer.getDouble(bytes, offset);
+            default:
+                return null;
+        }
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index b277885..f38c3e5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 2826279..3815666 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index e12dd71..1598165 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index 7e97dfa..4198934 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 947dea1..478c652 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
new file mode 100644
index 0000000..7ed194c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MissingIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new MissingIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.MISSING_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<AMissing> missingSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        missingSerde.serialize(AMissing.MISSING, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
new file mode 100644
index 0000000..af50678
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NanIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NanIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NAN_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.NaN);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..a70bc7b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NegInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NegInfIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NEGINF_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.NEGATIVE_INFINITY);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index bfd5c98..6e03ef8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
new file mode 100644
index 0000000..5df5142
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NullIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NullIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NULL_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ANull> nullSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        nullSerde.serialize(ANull.NULL, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..a1f4f13
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PosInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new PosInfIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.POSINF_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.POSITIVE_INFINITY);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
index e5e19ca..4337f28 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
@@ -46,7 +46,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new ABinaryHexStringConstructorDescriptor.ABinaryConstructorEvaluator(args[0],
-                        ByteArrayBase64ParserFactory.INSTANCE, ctx);
+                        ByteArrayBase64ParserFactory.INSTANCE, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
index 720e9bc..9f9a5f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx);
+                return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx, sourceLoc);
             }
         };
     }
@@ -70,6 +71,7 @@
     }
 
     static class ABinaryConstructorEvaluator implements IScalarEvaluator {
+        private final SourceLocation sourceLoc;
         private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
         private final DataOutput out = resultStorage.getDataOutput();
         private final IPointable inputArg = new VoidPointable();
@@ -78,7 +80,9 @@
         private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
 
         public ABinaryConstructorEvaluator(IScalarEvaluatorFactory copyEvaluatorFactory,
-                IValueParserFactory valueParserFactory, IHyracksTaskContext context) throws HyracksDataException {
+                IValueParserFactory valueParserFactory, IHyracksTaskContext context, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            this.sourceLoc = sourceLoc;
             eval = copyEvaluatorFactory.createScalarEvaluator(context);
             byteArrayParser = valueParserFactory.createValueParser();
         }
@@ -102,11 +106,11 @@
                     byteArrayParser.parse(buffer, 0, buffer.length, out);
                     result.set(resultStorage);
                 } else {
-                    throw new TypeMismatchException(BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
                             ATypeTag.SERIALIZED_BINARY_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                 }
             } catch (IOException e) {
-                throw new InvalidDataFormatException(BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
+                throw new InvalidDataFormatException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
                         ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
             }
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
index 2f981b9..7ddb7e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ABooleanConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
index 5b0d250..0649fc8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
@@ -98,11 +98,11 @@
                                 circleSerde.serialize(aCircle, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index eb9b40f..9849cef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -93,7 +93,7 @@
 
                                 // the string to be parsed should be at least 8 characters: YYYYMMDD
                                 if (stringLength < 8) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                                 }
 
@@ -117,11 +117,12 @@
                                 dateSerde.serialize(aDate, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                                    ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index f202dbe..9b71d4b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -91,7 +91,7 @@
                                 int startOffset = utf8Ptr.getCharStartOffset();
                                 // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
                                 if (stringLength < 14) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                                 }
                                 // +1 if it is negative (-)
@@ -102,7 +102,7 @@
                                 if (serString[startOffset + timeOffset] != 'T') {
                                     timeOffset += 2;
                                     if (serString[startOffset + timeOffset] != 'T') {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                                     }
                                 }
@@ -117,11 +117,11 @@
                                 datetimeSerde.serialize(aDateTime, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
index 0e7cc34..f6c6f09 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
@@ -99,11 +99,11 @@
                                 dayTimeDurationSerde.serialize(aDayTimeDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (Exception e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
index 87df400..f9991db 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADoubleConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index df39306..f3d1ba8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -93,11 +93,11 @@
                                 durationSerde.serialize(aDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
index b6f045e..09f2594 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
@@ -100,11 +100,11 @@
                                 floatSerde.serialize(aFloat, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
index 9413cb4..ae91c46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
@@ -112,17 +112,17 @@
                                             && serString[offset + 2] == '6' && offset + 3 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                     }
                                     value = (short) (value - digit);
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -133,11 +133,11 @@
                                 int16Serde.serialize(aInt16, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
index 985d966..b4f1ae8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
@@ -111,17 +111,17 @@
                                             && serString[offset + 2] == '2' && offset + 3 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                     }
                                     value -= digit;
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -132,11 +132,11 @@
                                 int32Serde.serialize(aInt32, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
index 61a35b6..a41ec77 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     public FunctionIdentifier getIdentifier() {
                         return AInt64ConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
index e4a9d32..3132640 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
@@ -112,17 +112,17 @@
                                             && offset + 2 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                     }
                                     value = (byte) (value - digit);
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -133,11 +133,11 @@
                                 int8Serde.serialize(aInt8, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e1) {
-                            throw new InvalidDataFormatException(getIdentifier(), e1,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e1,
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
index 9309591..f103192 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
@@ -90,7 +90,8 @@
 
                         try {
                             if (bytes0[offset0] != bytes1[offset1]) {
-                                throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
+                                throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
+                                        bytes1[offset1]);
                             }
 
                             long intervalStart, intervalEnd;
@@ -110,18 +111,18 @@
                                     intervalEnd = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                     break;
                                 default:
-                                    throw new UnsupportedItemTypeException(getIdentifier(), bytes0[offset0]);
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), bytes0[offset0]);
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, intervalType.serialize());
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index e2a3614..b1a6d0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -114,7 +114,7 @@
                                 int startOffset = utf8Ptr.getCharStartOffset();
                                 intervalStart = ADateParserFactory.parseDatePart(bytes0, startOffset, stringLength);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -139,7 +139,7 @@
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -147,14 +147,14 @@
                             intervalEnd = cal.getChrononInDays(intervalEnd);
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index 3e61405..8d15bff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -116,7 +116,7 @@
                                 if (bytes0[startOffset + timeOffset] != 'T') {
                                     timeOffset += 2;
                                     if (argPtr0.getByteArray()[startOffset + timeOffset] != 'T') {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                     }
                                 }
@@ -125,7 +125,7 @@
                                 intervalStart += ATimeParserFactory.parseTimePart(bytes0, startOffset + timeOffset + 1,
                                         stringLength - timeOffset - 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -149,19 +149,19 @@
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index fa402bd..e74e77a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -111,7 +111,7 @@
                                 intervalStart = ATimeParserFactory.parseTimePart(bytes0, utf8Ptr.getCharStartOffset(),
                                         stringLength);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -121,7 +121,7 @@
 
                             if (bytes1[offset1] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
                                 if (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -137,14 +137,14 @@
                                 ADurationParserFactory.parseDuration(bytes1, utf8Ptr.getCharStartOffset(), stringLength,
                                         aDuration, ADurationParseOption.All);
                                 if (aDuration.getMonths() != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
                                         aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
@@ -155,14 +155,14 @@
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
index 0ef585c..39bb515 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
@@ -101,11 +101,11 @@
                                 lineSerde.serialize(aLine, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
index 3415f38..4b8b0bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
@@ -96,11 +96,11 @@
                                 point3DSerde.serialize(aPoint3D, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POINT3D_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
index 24045b9..0a7c381 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
@@ -92,11 +92,11 @@
                                 pointSerde.serialize(aPoint, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
index eb139b0..f88713c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
@@ -86,7 +86,7 @@
                                 String s = utf8Ptr.toString();
                                 String[] points = WS.split(s.trim());
                                 if (points.length <= 2) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
                                 out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
@@ -98,11 +98,11 @@
                                 }
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
index ccaf7b8..69cd147 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
@@ -101,17 +101,17 @@
                                 } else if (aPoint[0].getX() < aPoint[1].getX() && aPoint[0].getY() < aPoint[1].getY()) {
                                     aRectangle.setValue(aPoint[0], aPoint[1]);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                                 }
                                 rectangle2DSerde.serialize(aRectangle, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
index 00b5475..8f0182c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
@@ -45,7 +45,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return AStringConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index c338a35..38d8e05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -93,7 +93,7 @@
 
                                 // the string to be parsed should be at least 6 characters: hhmmss
                                 if (stringLength < 6) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
 
@@ -108,11 +108,11 @@
                                 timeSerde.serialize(aTime, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
index 64b9278..53c36bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
@@ -99,11 +99,12 @@
                                 uuidSerde.serialize(uuid, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_UUID_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                                    ATypeTag.SERIALIZED_UUID_TYPE_TAG);
                         }
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
index b95d512..60344fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
@@ -96,11 +96,11 @@
                                 yearMonthDurationSerde.serialize(aYearMonthDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
index 9b9bf71..b78eb54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -52,12 +53,14 @@
     protected static final byte[] FALSE = UTF8StringUtil.writeStringToBytes("false");
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
 
-    protected AbstractBooleanConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractBooleanConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         inputArg = new VoidPointable();
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
@@ -70,7 +73,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
         }
     }
 
@@ -87,10 +90,10 @@
             } else if (UTF8_BINARY_CMP.compare(bytes, startOffset + 1, len - 1, FALSE, 0, FALSE.length) == 0) {
                 setBoolean(result, false);
             } else {
-                throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
index 456735f..5354fc0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -46,14 +47,16 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final IPointable inputArg;
     protected final AMutableDouble aDouble;
     protected final UTF8StringPointable utf8Ptr;
 
-    protected AbstractDoubleConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractDoubleConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
         inputArg = new VoidPointable();
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
         }
     }
 
@@ -87,12 +90,12 @@
                 handleUparseableString(result);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
     protected void handleUparseableString(IPointable result) throws HyracksDataException {
-        throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+        throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
     }
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
index 2f9470d..a49c1ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -46,14 +47,16 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final AMutableInt64 aInt64;
     protected final UTF8StringPointable utf8Ptr;
 
-    protected AbstractInt64ConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractInt64ConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         inputArg = new VoidPointable();
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
         }
     }
 
@@ -88,12 +91,12 @@
                 handleUnparseableString(result);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
     protected void handleUnparseableString(IPointable result) throws HyracksDataException {
-        throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+        throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_INT64_TYPE_TAG);
     }
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
index 52d3bf6..9f2dc2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -46,17 +47,19 @@
 public abstract class AbstractStringConstructorEvaluator implements IScalarEvaluator {
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final UTF8StringBuilder builder;
     protected final GrowableArray baaos;
 
-    protected AbstractStringConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractStringConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
+        this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
         inputArg = new VoidPointable();
-        this.inputEval = inputEval;
         builder = new UTF8StringBuilder();
         baaos = new GrowableArray();
     }
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
@@ -156,7 +159,7 @@
                 case MULTISET:
                 case UUID:
                 default:
-                    throw new UnsupportedTypeException(getIdentifier(), serString[offset]);
+                    throw new UnsupportedTypeException(sourceLoc, getIdentifier(), serString[offset]);
             }
             builder.finish();
             out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
index 9c38fbc..07dae0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -39,8 +40,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
     public AbstractBinaryStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
index af0a819..2181aa6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -53,9 +54,12 @@
 
     // Function ID, for error reporting.
     private final FunctionIdentifier funcID;
+    private final SourceLocation sourceLoc;
 
     public AbstractBinaryStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        this.sourceLoc = sourceLoc;
         this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
         this.evalRight = evalRightFactory.createScalarEvaluator(context);
         this.funcID = funcID;
@@ -80,10 +84,10 @@
 
         // Type check.
         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[offset0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[offset0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
 
         // Sets StringUTF8Pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
index ebe1273..a562a05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -41,8 +42,9 @@
     private AMutableInt32 resultValue = new AMutableInt32(0);
 
     public AbstractBinaryStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
index 7cdf673..04a0349 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.GrowableArray;
@@ -39,8 +40,9 @@
     protected final UTF8StringBuilder resultBuilder = new UTF8StringBuilder();
 
     public AbstractBinaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
index 936415d..b6cec11 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -53,7 +53,6 @@
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.runtime.exceptions.UnderflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -95,8 +94,7 @@
     abstract protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException;
 
     @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
-            throws AlgebricksException {
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
@@ -104,80 +102,102 @@
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
                 return new IScalarEvaluator() {
-                    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-                    private DataOutput out = resultStorage.getDataOutput();
-                    private IPointable argPtr0 = new VoidPointable();
-                    private IPointable argPtr1 = new VoidPointable();
-                    private IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
-                    private IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
-                    private double[] operandsFloating = new double[args.length];
-                    private long[] operandsInteger = new long[args.length];
-                    private int resultType;
-                    static protected final int typeInt8 = 1;
-                    static protected final int typeInt16 = 2;
-                    static protected final int typeInt32 = 3;
-                    static protected final int typeInt64 = 4;
-                    static protected final int typeFloat = 5;
-                    static protected final int typeDouble = 6;
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private final DataOutput out = resultStorage.getDataOutput();
+                    private final IPointable argPtr0 = new VoidPointable();
+                    private final IPointable argPtr1 = new VoidPointable();
+                    private final IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
+                    private final double[] operandsFloating = new double[args.length];
+                    private final long[] operandsInteger = new long[args.length];
 
-                    protected AMutableFloat aFloat = new AMutableFloat(0);
-                    protected AMutableDouble aDouble = new AMutableDouble(0);
-                    protected AMutableInt64 aInt64 = new AMutableInt64(0);
-                    protected AMutableInt32 aInt32 = new AMutableInt32(0);
-                    protected AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-                    protected AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    private final AMutableFloat aFloat = new AMutableFloat(0);
+                    private final AMutableDouble aDouble = new AMutableDouble(0);
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private final AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private final AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private final AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
 
-                    protected AMutableDuration aDuration = new AMutableDuration(0, 0);
-                    protected AMutableDate aDate = new AMutableDate(0);
-                    protected AMutableTime aTime = new AMutableTime(0);
-                    protected AMutableDateTime aDatetime = new AMutableDateTime(0);
+                    private final AMutableDuration aDuration = new AMutableDuration(0, 0);
+                    private final AMutableDate aDate = new AMutableDate(0);
+                    private final AMutableTime aTime = new AMutableTime(0);
+                    private final AMutableDateTime aDatetime = new AMutableDateTime(0);
 
-                    private ATypeTag typeTag;
                     @SuppressWarnings("rawtypes")
-                    private ISerializerDeserializer serde;
+                    private final ISerializerDeserializer int8Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int16Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int32Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int64Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer floatSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer dateSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer timeSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer dateTimeSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer durationSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
 
-                    @SuppressWarnings("unchecked")
                     @Override
+                    @SuppressWarnings("unchecked")
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-                        resultStorage.reset();
-                        resultType = 0;
-                        int currentType;
                         evalLeft.evaluate(tuple, argPtr0);
                         evalRight.evaluate(tuple, argPtr1);
 
-                        for (int i = 0; i < args.length; i++) {
+                        resultStorage.reset();
+
+                        ATypeTag argTypeMax = null;
+
+                        for (int i = 0; i < 2; i++) {
                             IPointable argPtr = i == 0 ? argPtr0 : argPtr1;
                             byte[] bytes = argPtr.getByteArray();
                             int offset = argPtr.getStartOffset();
 
-                            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+                            ATypeTag currentType;
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
                             switch (typeTag) {
                                 case TINYINT:
-                                    currentType = typeInt8;
+                                    currentType = ATypeTag.TINYINT;
                                     operandsInteger[i] = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case SMALLINT:
-                                    currentType = typeInt16;
+                                    currentType = ATypeTag.SMALLINT;
                                     operandsInteger[i] = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case INTEGER:
-                                    currentType = typeInt32;
+                                    currentType = ATypeTag.INTEGER;
                                     operandsInteger[i] = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case BIGINT:
-                                    currentType = typeInt64;
+                                    currentType = ATypeTag.BIGINT;
                                     operandsInteger[i] = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case FLOAT:
-                                    currentType = typeFloat;
+                                    currentType = ATypeTag.FLOAT;
                                     operandsFloating[i] = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
                                     break;
                                 case DOUBLE:
-                                    currentType = typeDouble;
+                                    currentType = ATypeTag.DOUBLE;
                                     operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
                                     break;
                                 case DATE:
@@ -186,11 +206,11 @@
                                 case DURATION:
                                 case YEARMONTHDURATION:
                                 case DAYTIMEDURATION:
-                                    evaluateTemporalArthmeticOperation(typeTag);
+                                    evaluateTemporalArithmeticOperation(typeTag);
                                     result.set(resultStorage);
                                     return;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), i, bytes[offset],
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                             ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
@@ -201,85 +221,76 @@
                                             ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                             }
 
-                            if (resultType < currentType) {
-                                resultType = currentType;
+                            if (i == 0 || currentType.ordinal() > argTypeMax.ordinal()) {
+                                argTypeMax = currentType;
                             }
                         }
 
+                        ATypeTag resultType = getNumericResultType(argTypeMax);
+
                         long lres;
                         double dres;
                         switch (resultType) {
-                            case typeInt8:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT8);
+                            case TINYINT:
                                 lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
                                 if (lres > Byte.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
                                 if (lres < Byte.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 aInt8.setValue((byte) lres);
-                                serde.serialize(aInt8, out);
+                                int8Serde.serialize(aInt8, out);
                                 break;
-                            case typeInt16:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT16);
+                            case SMALLINT:
                                 lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
                                 if (lres > Short.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
                                 if (lres < Short.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 aInt16.setValue((short) lres);
-                                serde.serialize(aInt16, out);
+                                int16Serde.serialize(aInt16, out);
                                 break;
-                            case typeInt32:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT32);
+                            case INTEGER:
                                 lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
                                 if (lres > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
                                 if (lres < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 aInt32.setValue((int) lres);
-                                serde.serialize(aInt32, out);
+                                int32Serde.serialize(aInt32, out);
                                 break;
-                            case typeInt64:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT64);
+                            case BIGINT:
                                 lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
                                 aInt64.setValue(lres);
-                                serde.serialize(aInt64, out);
+                                int64Serde.serialize(aInt64, out);
                                 break;
-                            case typeFloat:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
+                            case FLOAT:
                                 dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
                                 if (dres > Float.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
                                 if (dres < -Float.MAX_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 aFloat.setValue((float) dres);
-                                serde.serialize(aFloat, out);
+                                floatSerde.serialize(aFloat, out);
                                 break;
-                            case typeDouble:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
-                                aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
-                                serde.serialize(aDouble, out);
+                            case DOUBLE:
+                                dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
+                                aDouble.setValue(dres);
+                                doubleSerde.serialize(aDouble, out);
                                 break;
                         }
                         result.set(resultStorage);
                     }
 
                     @SuppressWarnings("unchecked")
-                    private void evaluateTemporalArthmeticOperation(ATypeTag leftType) throws HyracksDataException {
+                    private void evaluateTemporalArithmeticOperation(ATypeTag leftType) throws HyracksDataException {
                         byte[] bytes1 = argPtr1.getByteArray();
                         int offset1 = argPtr1.getStartOffset();
                         ATypeTag rightType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
@@ -288,10 +299,7 @@
 
                         if (rightType == leftType) {
 
-                            serde = SerializerDeserializerProvider.INSTANCE
-                                    .getSerializerDeserializer(BuiltinType.ADURATION);
-
-                            long leftChronon = 0, rightChronon = 0, dayTime = 0;
+                            long leftChronon = 0, rightChronon = 0, dayTime;
 
                             int yearMonth = 0;
 
@@ -301,7 +309,6 @@
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
                                     rightChronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
-
                                     break;
                                 case TIME:
                                     leftChronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -323,26 +330,25 @@
                                             ADayTimeDurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
                                     break;
                                 default:
-                                    throw new UnsupportedTypeException(getIdentifier(), bytes1[offset1]);
+                                    throw new UnsupportedTypeException(sourceLoc, getIdentifier(), bytes1[offset1]);
                             }
 
                             dayTime = evaluateTimeInstanceArithmetic(leftChronon, rightChronon);
 
                             aDuration.setValue(yearMonth, dayTime);
-
-                            serde.serialize(aDuration, out);
+                            durationSerde.serialize(aDuration, out);
 
                         } else {
                             long chronon = 0, dayTime = 0;
                             int yearMonth = 0;
                             ATypeTag resultType = null;
+                            ISerializerDeserializer serde = null;
 
                             boolean isTimeOnly = false;
 
                             switch (leftType) {
                                 case TIME:
-                                    serde = SerializerDeserializerProvider.INSTANCE
-                                            .getSerializerDeserializer(BuiltinType.ATIME);
+                                    serde = timeSerde;
                                     chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                     isTimeOnly = true;
                                     resultType = ATypeTag.TIME;
@@ -357,20 +363,18 @@
                                                     ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1);
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case DATE:
-                                    serde = SerializerDeserializerProvider.INSTANCE
-                                            .getSerializerDeserializer(BuiltinType.ADATE);
+                                    serde = dateSerde;
                                     resultType = ATypeTag.DATE;
                                     chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
                                 case DATETIME:
                                     if (leftType == ATypeTag.DATETIME) {
-                                        serde = SerializerDeserializerProvider.INSTANCE
-                                                .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                        serde = dateTimeSerde;
                                         resultType = ATypeTag.DATETIME;
                                         chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                     }
@@ -389,8 +393,8 @@
                                                     offset1 + 1);
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case YEARMONTHDURATION:
@@ -398,21 +402,19 @@
                                             AYearMonthDurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1);
                                     switch (rightType) {
                                         case DATETIME:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                            serde = dateTimeSerde;
                                             resultType = ATypeTag.DATETIME;
                                             chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                             break;
                                         case DATE:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATE);
+                                            serde = dateSerde;
                                             resultType = ATypeTag.DATE;
                                             chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case DURATION:
@@ -425,34 +427,31 @@
                                     }
                                     switch (rightType) {
                                         case DATETIME:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                            serde = dateTimeSerde;
                                             resultType = ATypeTag.DATETIME;
                                             chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                             break;
                                         case DATE:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATE);
+                                            serde = dateSerde;
                                             resultType = ATypeTag.DATE;
                                             chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                                             break;
                                         case TIME:
                                             if (yearMonth == 0) {
-                                                serde = SerializerDeserializerProvider.INSTANCE
-                                                        .getSerializerDeserializer(BuiltinType.ATIME);
+                                                serde = timeSerde;
                                                 resultType = ATypeTag.TIME;
                                                 chronon = ATimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                                 isTimeOnly = true;
                                                 break;
                                             }
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 default:
-                                    throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+                                    throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
                                             bytes1[offset1]);
                             }
 
@@ -477,7 +476,7 @@
                                     serde.serialize(aDatetime, out);
                                     break;
                                 default:
-                                    throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+                                    throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
                                             bytes1[offset1]);
                             }
                         }
@@ -486,4 +485,8 @@
             }
         };
     }
+
+    protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+        return argTypeMax;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
index 47c8d34..5de007b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -56,6 +57,7 @@
     private IScalarEvaluator eval3;
 
     private final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     private AMutableString resultBuffer = new AMutableString("");
     @SuppressWarnings("rawtypes")
@@ -69,12 +71,13 @@
 
     public AbstractQuadStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
             IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, IScalarEvaluatorFactory eval3,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
         this.eval3 = eval3.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
@@ -103,16 +106,16 @@
         resultStorage.reset();
         // Type check.
         if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes3[start3] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 3, bytes1[start3], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 3, bytes1[start3], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
 
         strPtr1st.set(bytes0, start0 + 1, len0);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
index 89bba67..c4bd804 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,10 +58,12 @@
 
     // Function ID, for error reporting.
     protected final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractStringStringStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
             IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, IScalarEvaluatorFactory eval3,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
+        this.sourceLoc = sourceLoc;
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
@@ -96,18 +99,18 @@
 
         // Type check.
         if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes3[start3] != ATypeTag.SERIALIZED_INT8_TYPE_TAG && bytes3[start3] != ATypeTag.SERIALIZED_INT16_TYPE_TAG
                 && bytes3[start3] != ATypeTag.SERIALIZED_INT32_TYPE_TAG
                 && bytes3[start3] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 3, bytes3[start3], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+            throw new TypeMismatchException(sourceLoc, funcID, 3, bytes3[start3], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT64_TYPE_TAG);
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
index b2da86d..5624bcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -36,9 +37,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
     public AbstractTripleStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
index 26563af..51d85f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -55,13 +56,16 @@
 
     // Function ID, for error reporting.
     protected final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractTripleStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0, IScalarEvaluatorFactory eval1,
-            IScalarEvaluatorFactory eval2, FunctionIdentifier funcID) throws HyracksDataException {
+            IScalarEvaluatorFactory eval2, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
@@ -87,13 +91,13 @@
 
         // Type check.
         if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
 
         // Sets argument UTF8Pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
index 1ee6c8b..5636a3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -38,9 +39,9 @@
     private final AMutableInt32 resultValue = new AMutableInt32(0);
 
     public AbstractTripleStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
index 3c73b36..031be7d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -37,9 +38,9 @@
     private final AMutableString resultValue = new AMutableString("");
 
     public AbstractTripleStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
index 241452a..00586d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 
 /**
@@ -31,8 +32,8 @@
 abstract class AbstractUnaryNumericDoubleFunctionEval extends AbstractUnaryNumericFunctionEval {
 
     public AbstractUnaryNumericDoubleFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, argEvalFactory, funcID);
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, argEvalFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
index 61cd6f6..d2a4164 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -92,11 +93,13 @@
 
     // The function identifier, used for error messages.
     private final FunctionIdentifier funcID;
+    private final SourceLocation sourceLoc;
 
     public AbstractUnaryNumericFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.argEval = argEvalFactory.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
@@ -126,7 +129,7 @@
             double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
             processDouble(val, result);
         } else {
-            throw new TypeMismatchException(funcID, 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+            throw new TypeMismatchException(sourceLoc, funcID, 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT64_TYPE_TAG, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
index a8853f1..fca1068 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -50,11 +51,13 @@
     private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private final DataOutput dataOutput = resultStorage.getDataOutput();
     private final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractUnaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.argEval = argEvalFactory.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -65,7 +68,8 @@
         int offset = argPtr.getStartOffset();
         byte inputTypeTag = argBytes[offset];
         if (inputTypeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, argBytes[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, argBytes[offset],
+                    ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         stringPtr.set(argBytes, offset + 1, argPtr.getLength() - 1);
         resultArray.reset();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
index e9f9c9e..e0d9bd3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -107,7 +107,7 @@
                                 continue;
                             }
                             if (bytes[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), i, bytes[offset],
                                         ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                             }
                             boolean argResult = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index b78bc5c..1833fcc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -61,7 +61,7 @@
         return BuiltinFunctions.ANY_COLLECTION_MEMBER;
     }
 
-    private static class AnyCollectionMemberEvalFactory implements IScalarEvaluatorFactory {
+    private class AnyCollectionMemberEvalFactory implements IScalarEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
 
@@ -95,8 +95,9 @@
 
                     if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                             && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.ANY_COLLECTION_MEMBER, 0, serList[offset],
-                                ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ANY_COLLECTION_MEMBER, 0,
+                                serList[offset], ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+                                ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                     }
 
                     try {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
index 1e65dc0..7f90973 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
@@ -68,7 +68,7 @@
 
                         if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                                 || data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                            throw new UnsupportedTypeException(getIdentifier(), data[offset]);
+                            throw new UnsupportedTypeException(sourceLoc, getIdentifier(), data[offset]);
                         }
                         result.set(inputArg);
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
index a59d03d..dbe2e05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
@@ -95,7 +95,8 @@
                                                 offset);
                                         break;
                                     default:
-                                        throw new UnsupportedTypeException(getIdentifier(), serOrderedList[offset]);
+                                        throw new UnsupportedTypeException(sourceLoc, getIdentifier(),
+                                                serOrderedList[offset]);
                                 }
                             }
                             // calculate length first
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
index 132c99e..8d49ae8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
@@ -91,11 +91,11 @@
 
                         // Type check: (point, double)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
index 6f79f97..9d49466 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
@@ -90,11 +90,11 @@
 
                         // type-check: (point, point)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index d61d6f2..71901c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -84,11 +84,11 @@
 
                         // type-check: (double, double)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
index 74f8757..bdb8b94 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
@@ -89,7 +89,7 @@
 
                             if (listBytes[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && listBytes[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, listBytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
@@ -107,16 +107,16 @@
                                         missingSerde.serialize(AMissing.MISSING, out);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(BuiltinFunctions.CREATE_POLYGON,
+                                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.CREATE_POLYGON,
                                             itemType.serialize());
                                 }
 
                             }
                             if (listAccessor.size() < 6) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             } else if (listAccessor.size() % 2 != 0) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
                             out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
index ad874f5..b758845 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
@@ -90,11 +90,11 @@
 
                         resultStorage.reset();
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index 0c55ef1..aa641ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -61,7 +61,7 @@
         return BuiltinFunctions.GET_ITEM;
     }
 
-    private static class GetItemEvalFactory implements IScalarEvaluatorFactory {
+    private class GetItemEvalFactory implements IScalarEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory listEvalFactory;
@@ -105,8 +105,8 @@
                             itemIndex = ATypeHierarchy.getIntegerValue(BuiltinFunctions.GET_ITEM.getName(), 0,
                                     indexBytes, indexOffset);
                         } else {
-                            throw new TypeMismatchException(BuiltinFunctions.GET_ITEM, 0, serOrderedList[offset],
-                                    ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
+                            throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_ITEM, 0,
+                                    serOrderedList[offset], ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                         }
 
                         if (itemIndex < 0 || itemIndex >= AOrderedListSerializerDeserializer
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
index 17f7a96..f4ba646 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
@@ -50,7 +50,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractUnaryStringStringEval(ctx, args[0],
-                        GetJobParameterByNameDescriptor.this.getIdentifier()) {
+                        GetJobParameterByNameDescriptor.this.getIdentifier(), sourceLoc) {
                     private byte[] result;
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
index 25b6c5d9..df875a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
@@ -82,7 +82,7 @@
                                     argPtr.getStartOffset() + 1);
                             if (argResult) {
                                 LOGGER.log(Level.ERROR, ctx.getTaskAttemptId() + " injecting failure");
-                                throw new RuntimeDataException(ErrorCode.INJECTED_FAILURE, getIdentifier());
+                                throw new RuntimeDataException(ErrorCode.INJECTED_FAILURE, sourceLoc, getIdentifier());
                             }
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
index 7bae18b..cbae79e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -83,7 +83,7 @@
 
                         if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, serList[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serList[offset],
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
index 13037a9..7c9736d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -86,7 +86,7 @@
                             ABoolean aResult = argRes ? ABoolean.FALSE : ABoolean.TRUE;
                             booleanSerde.serialize(aResult, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
index a7ba7b0..a0c9ae6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
@@ -59,7 +59,7 @@
 
         NumericACosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericACosDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericACosDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
index 02c5896..f20736c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericASinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericASinDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericASinDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
index d5a5c21..2939f49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericATanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericATanDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericATanDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
index d153a69..67d3e4d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
@@ -64,7 +64,7 @@
 
         NumericAbsEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericAbsDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericAbsDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index 3f52a47..f64b829 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -59,6 +59,6 @@
 
     @Override
     protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
deleted file mode 100644
index 0079a8a..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.evaluators.functions;
-
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.exceptions.OverflowException;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.google.common.math.LongMath;
-
-public class NumericCaretDescriptor extends AbstractNumericArithmeticEval {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new NumericCaretDescriptor();
-        }
-    };
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
-     */
-    @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        if (rhs > Integer.MAX_VALUE) {
-            throw new OverflowException(getIdentifier());
-        }
-        return LongMath.checkedPow(lhs, (int) rhs);
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
-     */
-    @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return Math.pow(lhs, rhs);
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
-     */
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return BuiltinFunctions.CARET;
-    }
-
-    @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
-            throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
-    }
-
-    @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
-    }
-
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
index 5d0eebb..407eb4d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericCeilingEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericCeilingDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericCeilingDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
index 7ef6bd4..1122bbe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericCosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericCosDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericCosDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
new file mode 100644
index 0000000..6539e20
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericDegreesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericDegreesDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+                        aDouble.setValue(Math.toDegrees(arg));
+                        serialize(aDouble, doubleSerde, resultPointable);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_DEGREES;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
new file mode 100644
index 0000000..56d90f7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NumericDivDescriptor extends AbstractNumericArithmeticEval {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericDivDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_DIV;
+    }
+
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) {
+        return lhs / rhs;
+    }
+
+    @Override
+    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+        if (rhs == 0) {
+            throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
+        }
+        if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
+        return lhs / rhs;
+    }
+
+    @Override
+    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly) {
+        throw new NotImplementedException("Divide operation is not defined for temporal types");
+    }
+
+    @Override
+    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 77c94bf..0c1a3eb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -18,21 +18,18 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class NumericDivideDescriptor extends AbstractNumericArithmeticEval {
-
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericDivideDescriptor();
@@ -45,14 +42,8 @@
     }
 
     @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        if (rhs == 0) {
-            throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
-        }
-        if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
-            throw new OverflowException(getIdentifier());
-        }
-        return lhs / rhs;
+    protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+        return argTypeMax.ordinal() < ATypeTag.FLOAT.ordinal() ? ATypeTag.DOUBLE : argTypeMax;
     }
 
     @Override
@@ -61,13 +52,17 @@
     }
 
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
-            throws HyracksDataException {
+    protected long evaluateInteger(long lhs, long rhs) {
+        throw new IllegalStateException();
+    }
+
+    @Override
+    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly) {
         throw new NotImplementedException("Divide operation is not defined for temporal types");
     }
 
     @Override
     protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
index c5aeaa1..3a51e48 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericExpEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericExpDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericExpDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
index fd001d8..7f1b9bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
@@ -64,7 +64,7 @@
 
         NumericFloorEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericFloorDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericFloorDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
index 1e9d2be..8cc3327 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericLnEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericLnDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericLnDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
index f5a35f6..8867ea7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericLogEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericLogDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericLogDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
index 777e447..a967162 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
@@ -55,11 +55,11 @@
     @Override
     protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
             throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
     }
 
     @Override
     protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index d907bf3..57de132 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -53,11 +53,11 @@
     @Override
     protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
             throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
     }
 
     @Override
     protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
new file mode 100644
index 0000000..a5312b9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.google.common.math.LongMath;
+
+public class NumericPowerDescriptor extends AbstractNumericArithmeticEval {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericPowerDescriptor();
+        }
+    };
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
+     */
+    @Override
+    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+        if (rhs > Integer.MAX_VALUE) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
+        return LongMath.checkedPow(lhs, (int) rhs);
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
+     */
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+        return Math.pow(lhs, rhs);
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+     */
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_POWER;
+    }
+
+    @Override
+    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+            throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+    }
+
+    @Override
+    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
new file mode 100644
index 0000000..a0c1a33
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericRadiansDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericRadiansDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+                        aDouble.setValue(Math.toRadians(arg));
+                        serialize(aDouble, doubleSerde, resultPointable);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_RADIANS;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
index 0ad0dbe..e355a05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
@@ -140,7 +140,7 @@
                             aDouble.setValue(val);
                             serde.serialize(aDouble, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
index 926ef49..00e7415 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
@@ -110,7 +110,7 @@
                         } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
                             return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
@@ -176,7 +176,7 @@
                                 serde.serialize(aDouble, out);
                             }
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
index 1d45b3c..ecb62e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
@@ -138,7 +138,7 @@
                             aDouble.setValue(Math.rint(val));
                             serde.serialize(aDouble, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
index 392dc0a..fcffdb8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSignEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSignDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSignDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
index 9b55844..08ddf71 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSinDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSinDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
index 2715639..cd48626 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSqrtEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSqrtDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSqrtDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
index 427e6eb..10b97e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
@@ -134,7 +134,7 @@
                                     operands[i] = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
                                     break;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), i, data[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), i, data[offset],
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                             ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
index 3b1772d..ade44a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericTanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericTanDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericTanDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
index fb52641..2e52d1e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
@@ -118,9 +118,9 @@
             } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
                 return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
             } else {
-                throw new TypeMismatchException(getIdentifier(), 1, bytes[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
+                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
             }
         }
 
@@ -178,10 +178,10 @@
                     serde.serialize(aDouble, out);
                 }
             } else {
-                throw new TypeMismatchException(getIdentifier(), 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT64_TYPE_TAG, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
-                        ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                        ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
             }
             result.set(resultStorage);
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
index 7f0b962..f60df2a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericUnaryMinusEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericUnaryMinusDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericUnaryMinusDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
index 7aea25c..b828c25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
@@ -103,7 +103,7 @@
                                 continue;
                             }
                             if (data[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), i, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), i, data[offset],
                                         ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                             }
                             boolean argResult = ABooleanSerializerDeserializer.getBoolean(data, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
new file mode 100644
index 0000000..fb99438
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RandomDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private final RandomHelper randomHelper = new RandomHelper(false);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        randomHelper.nextDouble(result);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANDOM;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
new file mode 100644
index 0000000..6dd5eac
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomWithSeedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RandomWithSeedDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final IPointable arg0 = new VoidPointable();
+
+                    private final RandomHelper randomHelper = new RandomHelper(true);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable resultPointable)
+                            throws HyracksDataException {
+                        eval0.evaluate(tuple, arg0);
+
+                        byte[] bytes = arg0.getByteArray();
+                        int offset = arg0.getStartOffset();
+                        ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+                        switch (tt) {
+                            case TINYINT:
+                            case SMALLINT:
+                            case INTEGER:
+                            case BIGINT:
+                            case FLOAT:
+                            case DOUBLE:
+                                randomHelper.setSeed(bytes, offset + 1, arg0.getLength() - 1);
+                                randomHelper.nextDouble(resultPointable);
+                                break;
+                            default:
+                                PointableHelper.setNull(resultPointable);
+                                break;
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANDOM_WITH_SEED;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
index f3d0bd8..1aec610 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
@@ -85,7 +85,7 @@
                                     int numOfPoints = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
 
                                     if (numOfPoints < 3) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                     }
                                     area = Math.abs(SpatialUtils.polygonArea(bytes, offset, numOfPoints));
@@ -118,7 +118,7 @@
                                     out.writeDouble(area);
                                     break;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
                                             ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index 9b47b92..ce9ab54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -128,19 +128,19 @@
                                 rectangleSerde.serialize(aRectangle, out);
                             } else {
                                 if (tag0 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 0, tag0,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tag0,
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                                 if (tag1 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 1, tag1,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, tag1,
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                                 if (tag2 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 2, tag2,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, tag2,
                                             ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                                 }
                                 if (tag3 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 3, tag3,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 3, tag3,
                                             ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                                 }
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
index ba47d3d..0fe73a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
@@ -93,11 +93,11 @@
                                             offset1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
                                     distance = Math.sqrt(Math.pow(x2 - x1, 2) + Math.pow(y2 - y1, 2));
                                 } else {
-                                    throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                             out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
index 851e1a4..cb213ae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -87,7 +87,7 @@
 
                             if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[listOffset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
@@ -113,7 +113,8 @@
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 utf8Len += UTF8StringUtil.getUTFLength(listBytes, itemOffset);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
index 46865cf..4f098fd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_CONTAINS) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_CONTAINS,
+                        sourceLoc) {
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
                         return UTF8StringPointable.contains(left, right, false);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
index 6422158..090c7d1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_ENDS_WITH) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_ENDS_WITH,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index 71a5742..1eb7a88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_EQUAL) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_EQUAL,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
index 10a67b6..2b3f43f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
@@ -50,7 +50,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringInitCapDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringInitCapDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
                             throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
index 70b3687..2c00ee5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -89,14 +89,14 @@
                         int listOffset = inputArgList.getStartOffset();
                         if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[listOffset],
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                         }
                         byte[] sepBytes = inputArgSep.getByteArray();
                         int sepOffset = inputArgSep.getStartOffset();
                         if (sepBytes[sepOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, sepBytes[sepOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, sepBytes[sepOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         int sepLen = UTF8StringUtil.getUTFLength(sepBytes, sepOffset + 1);
@@ -126,7 +126,8 @@
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 int currentSize = UTF8StringUtil.getUTFLength(listBytes, itemOffset);
                                 if (i != size - 1 && currentSize != 0) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
index bcf3d63..2595cd4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringLTrim2Descriptor.this.getIdentifier()) {
+                        StringLTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
index 7ec5a40..de635ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringLTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringLTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index b435025..9fd173a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -83,7 +83,7 @@
                                 result.setValue(len);
                                 int64Serde.serialize(result, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serString[offset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             resultPointable.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
index 4481328..fa1c0b1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
@@ -61,7 +61,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringLikeDescriptor.this.getIdentifier()) {
+                        StringLikeDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
                     private final RegExpMatcher.IRegExpPatternGenerator patternGenerator = new LikePatternGenerator();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index a9957f7..98c9a69 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -49,7 +49,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringLowerCaseDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringLowerCaseDescriptor.this.getIdentifier(),
+                        sourceLoc) {
 
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index e9ec211..b4311ea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -58,7 +58,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
-                        StringPositionDescriptor.this.getIdentifier()) {
+                        StringPositionDescriptor.this.getIdentifier(), sourceLoc) {
 
                     @Override
                     protected int compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
index 5d32bca..7f886ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringRTrim2Descriptor.this.getIdentifier()) {
+                        StringRTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
index 6ec708d..7c89832 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringRTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringRTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
index 7a23578..91e1ffa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
@@ -50,7 +50,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringRegExpContainsDescriptor.this.getIdentifier()) {
+                        StringRegExpContainsDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
index a5a79d9..21d038a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
-                        StringRegExpContainsWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpContainsWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
index 77eded9..83a6f39 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
@@ -51,7 +51,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringRegExpLikeDescriptor.this.getIdentifier()) {
+                        StringRegExpLikeDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
index 635b4a6..317545a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
@@ -49,7 +49,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
-                        StringRegExpLikeWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpLikeWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
index b39c473..62cf32d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
@@ -19,8 +19,6 @@
 
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.IOException;
-
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -59,7 +57,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
-                        StringRegExpPositionDescriptor.this.getIdentifier()) {
+                        StringRegExpPositionDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
index 0d910be..c85bd3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
@@ -57,7 +57,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringIntEval(ctx, args[0], args[1], args[2],
-                        StringRegExpPositionWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpPositionWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
index a1ea169..fc42f04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.IOException;
-
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,7 +48,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringStringEval(ctx, args[0], args[1], args[2],
-                        StringRegExpReplaceDescriptor.this.getIdentifier()) {
+                        StringRegExpReplaceDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
index f938c24..d901a22 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
@@ -50,7 +50,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractQuadStringStringEval(ctx, args[0], args[1], args[2], args[3],
-                        StringRegExpReplaceWithFlagsDescriptor.this.getIdentifier()) {
+                        StringRegExpReplaceWithFlagsDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
index 4672c75..90f4e35 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
@@ -87,7 +87,7 @@
                                 ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
                         // Checks repeatingTimes. It should be a non-negative value.
                         if (repeatingTimes < 0) {
-                            throw new RuntimeDataException(ErrorCode.NEGATIVE_VALUE, getIdentifier(), 1,
+                            throw new RuntimeDataException(ErrorCode.NEGATIVE_VALUE, sourceLoc, getIdentifier(), 1,
                                     repeatingTimes);
                         }
 
@@ -96,7 +96,7 @@
                         offset = argString.getStartOffset();
                         // Checks the type of the string argument.
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
index 098d98f..472cc25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -48,7 +48,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractTripleStringEval(ctx, args[0], args[1], args[2], getIdentifier()) {
+                return new AbstractTripleStringEval(ctx, args[0], args[1], args[2], getIdentifier(), sourceLoc) {
 
                     final StringReplacer replacer = new StringReplacer();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
index fcd8218..489255e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
@@ -48,7 +48,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringStringStringIntEval(ctx, args[0], args[1], args[2], args[3], getIdentifier()) {
+                return new AbstractStringStringStringIntEval(ctx, args[0], args[1], args[2], args[3], getIdentifier(),
+                        sourceLoc) {
 
                     final StringReplacer replacer = new StringReplacer();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
new file mode 100644
index 0000000..bcb8161
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringReverseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringReverseDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryStringStringEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    void process(UTF8StringPointable inputString, IPointable resultPointable) throws IOException {
+                        UTF8StringPointable.reverse(inputString, resultBuilder, resultArray);
+                        resultPointable.set(resultArray.getByteArray(), 0, resultArray.getLength());
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.STRING_REVERSE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
index 4cfb92d..5bdcb54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
@@ -97,7 +97,7 @@
                             int srcLen = argString.getLength();
                             // Type check for the first argument.
                             if (srcString[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, srcString[srcOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, srcString[srcOffset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -107,8 +107,8 @@
                             int patternLen = argPattern.getLength();
                             // Type check for the second argument.
                             if (patternString[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, patternString[patternOffset],
-                                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1,
+                                        patternString[patternOffset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
                             // Sets the UTF8 String pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
index aa43e66..360f156 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_STARTS_WITH) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_STARTS_WITH,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index 63d474d..8b0568b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -103,7 +103,7 @@
                                 }
                                 listBuilder.write(out, true);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serString[offset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
index 0136994..941f11f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringTrim2Descriptor.this.getIdentifier()) {
+                        StringTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
index bc4d150..391c8ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
index 53c7e1a..50f1d62 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
@@ -49,7 +49,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringUpperCaseDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringUpperCaseDescriptor.this.getIdentifier(),
+                        sourceLoc) {
 
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index 53f73ae..bda5b2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -86,19 +86,19 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset)
-                                - baseOffset;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
                         bytes = argString.getByteArray();
                         offset = argString.getStartOffset();
                         int len = argString.getLength();
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         string.set(bytes, offset + 1, len - 1);
                         array.reset();
                         try {
-                            UTF8StringPointable.substr(string, start, Integer.MAX_VALUE, builder, array);
+                            int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+                            UTF8StringPointable.substr(string, actualStart, Integer.MAX_VALUE, builder, array);
                         } catch (StringIndexOutOfBoundsException e) {
                             throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start);
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
index 92e0c52..eac2649 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
@@ -81,11 +81,11 @@
                         int patternLen = array1.getLength();
 
                         if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, src[srcOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, pattern[patternOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
index fe6631c..5ed13bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -83,11 +83,11 @@
                         int patternLen = array1.getLength();
 
                         if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, src[srcOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, pattern[patternOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index 148d0a4..dd1bd3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -93,8 +93,7 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset)
-                                - baseOffset;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset);
 
                         bytes = argLen.getByteArray();
                         offset = argLen.getStartOffset();
@@ -104,13 +103,14 @@
                         offset = argString.getStartOffset();
                         int length = argString.getLength();
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         string.set(bytes, offset + 1, length - 1);
                         array.reset();
                         try {
-                            UTF8StringPointable.substr(string, start, len, builder, array);
+                            int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+                            UTF8StringPointable.substr(string, actualStart, len, builder, array);
                         } catch (StringIndexOutOfBoundsException e) {
                             throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start + len - 1);
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
index 2762d61..8101bb5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
@@ -53,7 +53,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws IOException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
index fccd6d1..4a1bb70 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws HyracksDataException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
index 4fa33de..fecf528 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
@@ -53,7 +53,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws IOException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
index d044744..770f200 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
@@ -127,7 +127,7 @@
                                 break;
 
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                         ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
index 9559da4..595bb05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
@@ -54,7 +54,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @SuppressWarnings("unchecked")
                     private final ISerializerDeserializer<ANull> nullSerde =
                             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
index bcd8094..81a6adb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -38,37 +39,17 @@
     protected DataOutput dataOutput = resultStorage.getDataOutput();
     protected IPointable[] pointables;
     protected IScalarEvaluator[] evaluators;
+    protected SourceLocation sourceLoc;
 
-    public AbstractBinaryScalarEvaluator(final IHyracksTaskContext context,
-            final IScalarEvaluatorFactory[] evaluatorFactories) throws HyracksDataException {
+    public AbstractBinaryScalarEvaluator(IHyracksTaskContext context,
+            final IScalarEvaluatorFactory[] evaluatorFactories, SourceLocation sourceLoc) throws HyracksDataException {
         pointables = new IPointable[evaluatorFactories.length];
         evaluators = new IScalarEvaluator[evaluatorFactories.length];
         for (int i = 0; i < evaluators.length; ++i) {
             pointables[i] = new VoidPointable();
             evaluators[i] = evaluatorFactories[i].createScalarEvaluator(context);
         }
-    }
-
-    private static final String FIRST = "1st";
-    private static final String SECOND = "2nd";
-    private static final String THIRD = "3rd";
-    private static final String TH = "th";
-
-    protected String rankToString(int i) {
-        String prefix = "";
-        if (i >= 10) {
-            prefix = String.valueOf(i / 10);
-        }
-        switch (i % 10) {
-            case 1:
-                return prefix + FIRST;
-            case 2:
-                return prefix + SECOND;
-            case 3:
-                return prefix + THIRD;
-            default:
-                return String.valueOf(i) + TH;
-        }
+        this.sourceLoc = sourceLoc;
     }
 
     protected void checkTypeMachingThrowsIfNot(String title, ATypeTag[] expected, ATypeTag... actual)
@@ -77,7 +58,8 @@
             if (expected[i] != actual[i]) {
                 if (!ATypeHierarchy.canPromote(actual[i], expected[i])
                         && !ATypeHierarchy.canPromote(expected[i], actual[i])) {
-                    throw new TypeMismatchException(title, i, actual[i].serialize(), expected[i].serialize());
+                    throw new TypeMismatchException(sourceLoc, title, i, actual[i].serialize(),
+                            expected[i].serialize());
                 }
             }
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
index c88fe25..1637130 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -46,8 +47,8 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     public AbstractFindBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            int baseOffset, String functionName) throws HyracksDataException {
-        super(context, copyEvaluatorFactories);
+            int baseOffset, String functionName, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, copyEvaluatorFactories, sourceLoc);
         this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
index 6c84893..391df5e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -42,8 +43,8 @@
     private static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.INTEGER };
 
     public AbstractSubBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            int baseOffset, String functionName) throws HyracksDataException {
-        super(context, copyEvaluatorFactories);
+            int baseOffset, String functionName, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, copyEvaluatorFactories, sourceLoc);
         this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
index 907dfd3..59f3396 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
@@ -66,7 +66,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private final ListAccessor listAccessor = new ListAccessor();
                     private final byte[] metaBuffer = new byte[5];
@@ -87,7 +87,7 @@
                         byte typeTag = data[offset];
                         if (typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG
                                 && typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                         }
@@ -102,7 +102,8 @@
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 concatLength += ByteArrayPointable.getContentLength(data, itemOffset);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
index 2c53225..dc8edbe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
@@ -56,7 +56,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private AMutableInt64 result = new AMutableInt64(0);
                     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
index 307585d..f83f092 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) {
                         return 0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
index 243d344..fff7ca5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
@@ -62,7 +62,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
index 67d7b0f..35e7fba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
@@ -67,7 +67,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ABinary> binarySerde =
@@ -108,7 +108,7 @@
 
                             aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
                         } else {
-                            throw new UnsupportedItemTypeException(getIdentifier(), formatTag.serialize());
+                            throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), formatTag.serialize());
                         }
                         binarySerde.serialize(aBinary, dataOutput);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
index 11b68d3..c141e88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
@@ -66,7 +66,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private StringBuilder stringBuilder = new StringBuilder();
                     private final ByteArrayPointable byteArrayPtr = new ByteArrayPointable();
@@ -101,7 +101,7 @@
                                 Base64Printer.printBase64Binary(byteArrayPtr.getByteArray(),
                                         byteArrayPtr.getContentStartOffset(), lengthBinary, stringBuilder);
                             } else {
-                                throw new UnsupportedItemTypeException(getIdentifier(), arg1Tag.serialize());
+                                throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), arg1Tag.serialize());
                             }
                             dataOutput.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             writer.writeUTF8(stringBuilder.toString(), dataOutput);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
index 3b07c6c..c2fcdab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) {
                         return Integer.MAX_VALUE;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
index 961edbf..a9c4c46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
@@ -62,7 +62,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
index d641ea9..d624133 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessByIndexEvalFactory(args[0], args[1], recType);
+        return new FieldAccessByIndexEvalFactory(args[0], args[1], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
index d5918cb..3454c2e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -49,13 +50,15 @@
     private IScalarEvaluatorFactory fieldIndexEvalFactory;
     private int nullBitmapSize;
     private ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessByIndexEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fieldIndexEvalFactory, ARecordType recordType) {
+            IScalarEvaluatorFactory fieldIndexEvalFactory, ARecordType recordType, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fieldIndexEvalFactory = fieldIndexEvalFactory;
         this.recordType = recordType;
         this.nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -89,15 +92,15 @@
                     int offset = inputArg0.getStartOffset();
 
                     if (serRecord[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 0, serRecord[offset],
-                                ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 0,
+                                serRecord[offset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
                     eval1.evaluate(tuple, inputArg1);
                     byte[] indexBytes = inputArg1.getByteArray();
                     int indexOffset = inputArg1.getStartOffset();
                     if (indexBytes[indexOffset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 1, indexBytes[offset],
-                                ATypeTag.SERIALIZED_INT32_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 1,
+                                indexBytes[offset], ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                     }
                     fieldIndex = IntegerPointable.getInteger(indexBytes, indexOffset + 1);
                     fieldValueType = recordType.getFieldTypes()[fieldIndex];
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
index 4312cac..acd11c7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
@@ -41,7 +41,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessByNameEvalFactory(args[0], args[1]);
+        return new FieldAccessByNameEvalFactory(args[0], args[1], sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 2078921..774b427 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -42,11 +43,13 @@
 
     private IScalarEvaluatorFactory recordEvalFactory;
     private IScalarEvaluatorFactory fldNameEvalFactory;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessByNameEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fldNameEvalFactory) {
+            IScalarEvaluatorFactory fldNameEvalFactory, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fldNameEvalFactory = fldNameEvalFactory;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -75,7 +78,7 @@
                     int serRecordLen = inputArg0.getLength();
 
                     if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_NAME, 0,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.FIELD_ACCESS_BY_NAME, 0,
                                 serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
                     byte[] serFldName = inputArg1.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
index 5dd3e19..91b36cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
@@ -61,6 +61,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessNestedEvalFactory(args[0], recType, fldName);
+        return new FieldAccessNestedEvalFactory(args[0], recType, fldName, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 067a458..8fee257 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -44,6 +44,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -54,16 +55,17 @@
 
     private static final long serialVersionUID = 1L;
 
-    private IScalarEvaluatorFactory recordEvalFactory;
-    private ARecordType recordType;
-    private List<String> fieldPath;
+    private final IScalarEvaluatorFactory recordEvalFactory;
+    private final ARecordType recordType;
+    private final List<String> fieldPath;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessNestedEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType,
-            List<String> fldName) {
+            List<String> fldName, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.recordType = recordType;
         this.fieldPath = fldName;
-
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -116,8 +118,8 @@
                     int len = inputArg0.getLength();
 
                     if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_NESTED, 0, serRecord[start],
-                                ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.FIELD_ACCESS_NESTED, 0,
+                                serRecord[start], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
 
                     int subFieldIndex = -1;
@@ -139,8 +141,8 @@
                             subType = ((AUnionType) subType).getActualType();
                             byte serializedTypeTag = subType.getTypeTag().serialize();
                             if (serializedTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                                throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED.getName(),
-                                        serializedTypeTag);
+                                throw new UnsupportedTypeException(sourceLoc,
+                                        BuiltinFunctions.FIELD_ACCESS_NESTED.getName(), serializedTypeTag);
                             }
                             if (subType.getTypeTag() == ATypeTag.OBJECT) {
                                 recTypeInfos[pathIndex].reset((ARecordType) subType);
@@ -195,7 +197,8 @@
                         // type check
                         if (pathIndex < fieldPointables.length - 1
                                 && serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED, serRecord[start]);
+                            throw new UnsupportedTypeException(sourceLoc, BuiltinFunctions.FIELD_ACCESS_NESTED,
+                                    serRecord[start]);
                         }
                     }
 
@@ -229,8 +232,8 @@
                             return;
                         }
                         if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED.getName(),
-                                    serRecord[start]);
+                            throw new UnsupportedTypeException(sourceLoc,
+                                    BuiltinFunctions.FIELD_ACCESS_NESTED.getName(), serRecord[start]);
                         }
                     }
                     // emit the final result.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
index 4c07e2d..b25ba1e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new GetRecordFieldValueEvalFactory(args[0], args[1], recType);
+        return new GetRecordFieldValueEvalFactory(args[0], args[1], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 95fb91a..53dbaad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -46,12 +47,14 @@
     private IScalarEvaluatorFactory recordEvalFactory;
     private IScalarEvaluatorFactory fldNameEvalFactory;
     private final ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
     public GetRecordFieldValueEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fldNameEvalFactory, ARecordType recordType) {
+            IScalarEvaluatorFactory fldNameEvalFactory, ARecordType recordType, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fldNameEvalFactory = fldNameEvalFactory;
         this.recordType = recordType;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -86,7 +89,7 @@
                     int serRecordLen = inputArg0.getLength();
 
                     if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.GET_RECORD_FIELD_VALUE, 0,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_RECORD_FIELD_VALUE, 0,
                                 serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
index 57e98d0..03c26e3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new GetRecordFieldsEvalFactory(args[0], recType);
+        return new GetRecordFieldsEvalFactory(args[0], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
index 69ea1bc..16e24b4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -41,10 +42,13 @@
 
     private IScalarEvaluatorFactory recordEvalFactory;
     private final ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
-    public GetRecordFieldsEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType) {
+    public GetRecordFieldsEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType,
+            SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.recordType = recordType;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -68,7 +72,7 @@
                 int len = inputArg0.getLength();
 
                 if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.GET_RECORD_FIELDS, 0, data[offset],
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_RECORD_FIELDS, 0, data[offset],
                             ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                 }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
index 71a8291..0edd74b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
@@ -147,14 +147,14 @@
                         // Make sure we get a valid record
                         byte typeTag0 = argPtr0.getByteArray()[argPtr0.getStartOffset()];
                         if (typeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag0,
                                     ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                         }
 
                         // Make sure we get a valid list
                         byte typeTag1 = argPtr1.getByteArray()[argPtr1.getStartOffset()];
                         if (typeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, typeTag1,
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                         }
 
@@ -229,7 +229,8 @@
                                 }
 
                                 if (namePointable == null || valuePointable == null) {
-                                    throw new InvalidDataFormatException(getIdentifier(), "fields to be added");
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                            "fields to be added");
                                 }
 
                                 // Check that the field being added is a valid field
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
index 0a3ba36..401335c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
@@ -57,7 +57,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordConcatEvalFactory(args, argTypes, false);
+        return new RecordConcatEvalFactory(args, argTypes, false, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
index 82cc876..989f217 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,10 +57,14 @@
 
     private final boolean failOnArgTypeMismatch;
 
-    RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, boolean failOnArgTypeMismatch) {
+    private final SourceLocation sourceLoc;
+
+    RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, boolean failOnArgTypeMismatch,
+            SourceLocation sourceLoc) {
         this.args = args;
         this.argTypes = argTypes;
         this.failOnArgTypeMismatch = failOnArgTypeMismatch;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -161,7 +166,7 @@
                     returnNull = true;
                 } else if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
                     if (failOnArgTypeMismatch) {
-                        throw new TypeMismatchException(BuiltinFunctions.RECORD_CONCAT, i, typeTag,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.RECORD_CONCAT, i, typeTag,
                                 ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     } else {
                         returnNull = true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
index 77fe301..b51f66a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
@@ -56,7 +56,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordConcatEvalFactory(args, argTypes, true);
+        return new RecordConcatEvalFactory(args, argTypes, true, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index 6f8ba29..770f5bb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -120,7 +120,7 @@
                         argEvaluator.evaluate(tuple, argPtr);
                         byte inputTypeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
                         if (inputTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, inputTypeTag,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, inputTypeTag,
                                     ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                         }
                         recordVisitablePointable.set(argPtr);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
new file mode 100644
index 0000000..b57f01e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordRemoveDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_REMOVE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
new file mode 100644
index 0000000..d92c9a1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordRemoveEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final UTF8StringPointable fieldToRemovePointable = new UTF8StringPointable();
+    private final IBinaryComparator stringBinaryComparator = PointableHelper.createStringBinaryComparator();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final ARecordVisitablePointable openRecordPointable;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+
+    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType recordType) {
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg =
+                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        boolean returnNull = false;
+        eval0.evaluate(tuple, inputRecordPointable);
+        byte[] data = inputRecordPointable.getByteArray();
+        int offset = inputRecordPointable.getStartOffset();
+        byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval1.evaluate(tuple, fieldToRemovePointable);
+        data = fieldToRemovePointable.getByteArray();
+        offset = fieldToRemovePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        if (returnNull) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        evaluate();
+        result.set(resultStorage);
+    }
+
+    private void evaluate() throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
+            buildOutputRecord(inputRecord);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
+                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            }
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordPointable;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
index 909c74b..54db1e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
@@ -60,7 +60,8 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordRemoveFieldsEvalFactory(args[0], args[1], outputRecordType, inputRecType, inputListType);
+        return new RecordRemoveFieldsEvalFactory(args[0], args[1], outputRecordType, inputRecType, inputListType,
+                sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
index aeee36f..2d372f3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
@@ -44,6 +44,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,16 +57,17 @@
     private ARecordType requiredRecType;
     private ARecordType inputRecType;
     private AOrderedListType inputListType;
+    private final SourceLocation sourceLoc;
 
     public RecordRemoveFieldsEvalFactory(IScalarEvaluatorFactory inputRecordEvalFactory,
             IScalarEvaluatorFactory removeFieldPathsFactory, ARecordType requiredRecType, ARecordType inputRecType,
-            AOrderedListType inputListType) {
+            AOrderedListType inputListType, SourceLocation sourceLoc) {
         this.inputRecordEvalFactory = inputRecordEvalFactory;
         this.removeFieldPathsFactory = removeFieldPathsFactory;
         this.requiredRecType = requiredRecType;
         this.inputRecType = inputRecType;
         this.inputListType = inputListType;
-
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -98,13 +100,13 @@
 
                 byte inputTypeTag0 = inputArg0.getByteArray()[inputArg0.getStartOffset()];
                 if (inputTypeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.REMOVE_FIELDS, 0, inputTypeTag0,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.REMOVE_FIELDS, 0, inputTypeTag0,
                             ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                 }
 
                 byte inputTypeTag1 = inputArg1.getByteArray()[inputArg1.getStartOffset()];
                 if (inputTypeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.REMOVE_FIELDS, 1, inputTypeTag1,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.REMOVE_FIELDS, 1, inputTypeTag1,
                             ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                 }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
new file mode 100644
index 0000000..ac1b8e2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordRenameDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_RENAME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
new file mode 100644
index 0000000..3d46b69
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordRenameEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final UTF8StringPointable oldFieldNamePointable = new UTF8StringPointable();
+    private final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
+    private final IBinaryComparator stringBinaryComparator = PointableHelper.createStringBinaryComparator();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final IScalarEvaluator eval2;
+    private final ARecordVisitablePointable openRecordPointable;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+
+    RecordRenameEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
+            ARecordType recordType) {
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        this.eval2 = eval2;
+        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg =
+                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        boolean returnNull = false;
+        eval0.evaluate(tuple, inputRecordPointable);
+        byte[] data = inputRecordPointable.getByteArray();
+        int offset = inputRecordPointable.getStartOffset();
+        byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval1.evaluate(tuple, oldFieldNamePointable);
+        data = oldFieldNamePointable.getByteArray();
+        offset = oldFieldNamePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval2.evaluate(tuple, newFieldNamePointable);
+        data = newFieldNamePointable.getByteArray();
+        offset = newFieldNamePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        if (returnNull) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        evaluate();
+        result.set(resultStorage);
+    }
+
+    private void evaluate() throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
+            buildOutputRecord(inputRecord);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
+                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            } else {
+                outRecordBuilder.addField(newFieldNamePointable, fieldValues.get(i));
+            }
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordPointable;
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java
new file mode 100644
index 0000000..5890d7a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordUnwrapDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordUnwrapDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                return new RecordUnwrapEvaluator(eval0, recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_UNWRAP;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java
new file mode 100644
index 0000000..97bc01a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordUnwrapEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final IScalarEvaluator eval0;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private ARecordVisitablePointable openRecordVisitablePointable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+
+    RecordUnwrapEvaluator(IScalarEvaluator eval0, ARecordType recordType) {
+        this.eval0 = eval0;
+        openRecordVisitablePointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg = new Triple<>(openRecordVisitablePointable,
+                        openRecordVisitablePointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        eval0.evaluate(tuple, inputRecordPointable);
+        final byte[] data = inputRecordPointable.getByteArray();
+        final int offset = inputRecordPointable.getStartOffset();
+        final byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        final ARecordVisitablePointable inputRecordVisitablePointable = getInputRecordVisitablePointable();
+        final List<IVisitablePointable> recValues = inputRecordVisitablePointable.getFieldValues();
+        if (recValues.size() != 1) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        writeValue(recValues.get(0));
+        result.set(resultStorage);
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordVisitablePointable;
+    }
+
+    private void writeValue(IVisitablePointable value) throws HyracksDataException {
+        try {
+            resultOutput.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
index 0a1bcf5..6edfb48 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
@@ -51,7 +51,7 @@
 
                 return new IScalarEvaluator() {
 
-                    protected final IntervalLogic il = new IntervalLogic();
+                    protected final IntervalLogic il = new IntervalLogic(sourceLoc);
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     private DataOutput out = resultStorage.getDataOutput();
                     private TaggedValuePointable argPtr0 =
@@ -78,12 +78,12 @@
 
                         byte typeTag0 = argPtr0.getTag();
                         if (typeTag0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag0,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         byte typeTag1 = argPtr0.getTag();
                         if (typeTag1 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, typeTag1,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
 
@@ -91,7 +91,7 @@
                         argPtr1.getValue(interval1);
 
                         if (typeTag0 != typeTag1) {
-                            throw new IncompatibleTypeException(getIdentifier(), typeTag0, typeTag1);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), typeTag0, typeTag1);
                         }
 
                         ABoolean res = compareIntervals(il, interval0, interval1) ? ABoolean.TRUE : ABoolean.FALSE;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
index 38d890c..89865ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
@@ -95,12 +95,12 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
 
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -109,7 +109,7 @@
                                     utf8Ptr.getCharStartOffset());
 
                             if (!calInstance.validateTimeZone(timezone)) {
-                                throw new InvalidDataFormatException(getIdentifier(), "timezone");
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), "timezone");
                             }
 
                             long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
index 8c09e47..b57961a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
@@ -93,12 +93,12 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             }
 
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -106,7 +106,7 @@
                             int timezone = ATimeParserFactory.parseTimezonePart(bytes1, utf8Ptr.getCharStartOffset());
 
                             if (!calInstance.validateTimeZone(timezone)) {
-                                throw new InvalidDataFormatException(getIdentifier(), "timezone");
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), "timezone");
                             }
 
                             int chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
index 3dab641..6244087 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
@@ -94,12 +94,12 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
 
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index 867d030..5ee87fa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -111,12 +111,12 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
 
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
index cfac2e9..cc449b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
@@ -84,7 +84,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
index f70a6e2..43764ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
@@ -93,11 +93,11 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
index 87e83d7..9e16fc3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
@@ -104,7 +104,7 @@
                                 aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1));
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
index abf4016..47705a2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
@@ -105,7 +105,7 @@
                                 aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1) * 1000l);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
index 7bb4265..59006ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
@@ -94,7 +94,7 @@
                             chronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1)
                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
index cb26c58..172e2e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
@@ -83,17 +83,17 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
                         if ((ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1) != 0)
                                 || (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0)) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
index 9b29ca8..8739fe5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
@@ -84,11 +84,11 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
index d8be601..7761fb8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
@@ -85,7 +85,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         long chrononStart = AIntervalSerializerDeserializer.getIntervalStart(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
index f1987ca..b81ae06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
@@ -100,7 +100,7 @@
                                 aDuration.setValue(0, AInt64SerializerDeserializer.getLong(bytes, offset + 1));
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
index 9410e6d..733c349 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
@@ -84,7 +84,7 @@
                         int offset = argPtr0.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index 4bd3ede..7a15151 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -62,7 +62,7 @@
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
                 return new IScalarEvaluator() {
 
-                    protected final IntervalLogic il = new IntervalLogic();
+                    protected final IntervalLogic il = new IntervalLogic(sourceLoc);
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     private DataOutput out = resultStorage.getDataOutput();
                     private TaggedValuePointable argPtr0 =
@@ -100,7 +100,8 @@
                             byte intervalType1 = interval1.getType();
 
                             if (intervalType0 != intervalType1) {
-                                throw new IncompatibleTypeException(getIdentifier(), intervalType0, intervalType1);
+                                throw new IncompatibleTypeException(sourceLoc, getIdentifier(), intervalType0,
+                                        intervalType1);
                             }
 
                             if (il.overlaps(interval0, interval1) || il.overlappedBy(interval0, interval1)
@@ -115,10 +116,10 @@
                             result.set(resultStorage);
                             return;
                         } else if (type0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, type0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, type0,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         } else {
-                            throw new IncompatibleTypeException(getIdentifier(), type0, type1);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), type0, type1);
                         }
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
index fd58d6c..03faa96 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
@@ -81,7 +81,7 @@
                         byte[] bytes0 = argPtr0.getByteArray();
                         int offset0 = argPtr0.getStartOffset();
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         aYearMonthDuration.setMonths(ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
index 5694c27..ac2e06a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
@@ -117,7 +117,7 @@
                                 chrononToBin = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -127,7 +127,8 @@
                         ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
 
                         if (type0 != type1) {
-                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
+                                    bytes1[offset1]);
                         }
 
                         long chrononToStart = 0;
@@ -143,7 +144,7 @@
                                 chrononToStart = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -169,11 +170,11 @@
                                         + ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
 
                                 if (binIndex > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
 
                                 if (binIndex < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
 
                                 break;
@@ -209,7 +210,7 @@
                                 break;
                             case TIME:
                                 if (yearMonth != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                                 binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart,
@@ -224,7 +225,7 @@
                                         yearMonth * ((int) binIndex + 1), dayTime * (binIndex + 1), false);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
index 8f447d0..db0046c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
@@ -23,18 +23,23 @@
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 
 public class IntervalLogic implements Serializable {
 
     private static final long serialVersionUID = 1L;
-    private final ComparisonHelper ch = new ComparisonHelper();
+    private final ComparisonHelper ch;
     private final transient IPointable s1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable s2 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e2 = VoidPointable.FACTORY.createPointable();
 
+    public IntervalLogic(SourceLocation sourceLoc) {
+        ch = new ComparisonHelper(sourceLoc);
+    }
+
     public boolean validateInterval(AIntervalPointable ip1) throws HyracksDataException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
index c3bf58d..8e01e11 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
@@ -80,7 +80,7 @@
                         byte[] bytes = argPtr0.getByteArray();
                         int offset = argPtr0.getStartOffset();
                         if (bytes[offset] != ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
                         aInt64.setValue(ADayTimeDurationSerializerDeserializer.getDayTime(bytes, offset + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 3d595c3..5bfd78e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -81,7 +81,7 @@
                         int offset = argPtr0.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
                         aInt64.setValue(AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, offset + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
index 9710e3f..c8f73b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
@@ -125,7 +125,7 @@
                                 intervalStart = intervalStart * GregorianCalendarSystem.CHRONON_OF_DAY;
                             }
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
 
@@ -134,7 +134,8 @@
                         int offset1 = argPtr1.getStartOffset();
                         ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
                         if (intervalTypeTag != bytes1[offset1]) {
-                            throw new IncompatibleTypeException(getIdentifier(), intervalTypeTag, bytes1[offset1]);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), intervalTypeTag,
+                                    bytes1[offset1]);
                         }
 
                         long anchorTime;
@@ -150,7 +151,7 @@
                                 anchorTime = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -178,11 +179,11 @@
                                         + ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
 
                                 if (firstBinIndex > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
 
                                 if (firstBinIndex < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 break;
 
@@ -195,7 +196,7 @@
                                         + ((totalChronon < 0 && totalChronon % dayTime != 0) ? -1 : 0);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 2, bytes2[offset2],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, bytes2[offset2],
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
@@ -233,7 +234,7 @@
 
                             } else if (intervalTypeTag == ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
                                 if (yearMonth != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -248,7 +249,7 @@
 
                                 if (binStartChronon < 0 || binStartChronon >= GregorianCalendarSystem.CHRONON_OF_DAY) {
                                     // avoid the case where a time bin is before 00:00:00 or no early than 24:00:00
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -272,7 +273,7 @@
                                     }
 
                                     if (binEndChronon < binStartChronon) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                     }
                                 }
@@ -292,7 +293,7 @@
                                     binOffset++;
                                 } while (binEndChronon < intervalEnd);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
index f8d5cf7..da8d732 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -104,11 +104,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
@@ -136,7 +136,8 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         aDate.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
                         dateSerde.serialize(aDate, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
index 8bb1a20..ab19f9e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -96,11 +96,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
@@ -127,7 +127,7 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         aDateTime.setValue(aInt64.getLongValue());
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
index 6b9488d..af85b5b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -96,11 +96,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
@@ -127,7 +127,8 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
                         aTime.setValue((int) aInt64.getLongValue());
                         timeSerde.serialize(aTime, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
index 98fe013..c0b155d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -89,11 +89,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             long chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
index 5d2f088..066bef0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -89,11 +89,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
index 48a5382..2ad3642 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -88,11 +88,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
index 273cba4..3010540 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
@@ -89,7 +89,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
index 3dc449d..a069347 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
@@ -82,7 +82,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         long dateChronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
index 2dc57c5..4003195 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
@@ -80,7 +80,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
index 34fedb6..b8adc06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
@@ -86,7 +86,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
index 0909764..c1932f6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
@@ -82,7 +82,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
                         long timeChronon = ATimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
index 5748956..f7407c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
@@ -83,17 +83,17 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
                         if ((ADurationSerializerDeserializer.getDayTime(bytes0, offset0 + 1) != 0)
                                 || (ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1) != 0)) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
new file mode 100644
index 0000000..7ed1ce5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions.utils;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.security.SecureRandom;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.DataUtils;
+import org.apache.hyracks.data.std.util.GrowableArray;
+
+public final class RandomHelper {
+
+    private final SecureRandom random = new SecureRandom();
+
+    private final GrowableArray seed;
+
+    private final AMutableDouble aDouble = new AMutableDouble(0);
+
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer doubleSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput dataOutput = resultStorage.getDataOutput();
+
+    public RandomHelper(boolean withSeed) {
+        seed = withSeed ? new GrowableArray(8) : null;
+    }
+
+    public void setSeed(byte[] bytes, int offset, int length) throws HyracksDataException {
+        if (seed == null) {
+            throw new IllegalStateException();
+        }
+
+        boolean sameSeed =
+                seed.getLength() == length && DataUtils.equalsInRange(seed.getByteArray(), 0, bytes, offset, length);
+
+        if (!sameSeed) {
+            try {
+                seed.reset();
+                seed.append(bytes, offset, length);
+                random.setSeed(seed.getByteArray());
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+
+    public void nextDouble(IPointable resultPointable) throws HyracksDataException {
+        aDouble.setValue(random.nextDouble());
+        resultStorage.reset();
+        doubleSerde.serialize(aDouble, dataOutput);
+        resultPointable.set(resultStorage);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
index 5feb87f..503958b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
@@ -23,19 +23,23 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IncompatibleTypeException extends RuntimeDataException {
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
-    public IncompatibleTypeException(FunctionIdentifier fid, byte typeTagLeft, byte typeTagRight) {
-        super(ErrorCode.TYPE_INCOMPATIBLE, fid.getName(),
+    public IncompatibleTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
     }
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
-    public IncompatibleTypeException(String functionName, byte typeTagLeft, byte typeTagRight) {
-        super(ErrorCode.TYPE_INCOMPATIBLE, functionName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+    public IncompatibleTypeException(SourceLocation sourceLoc, String functionName, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, functionName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
index acb0495..3350fed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
@@ -23,20 +23,22 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class InvalidDataFormatException extends RuntimeDataException {
 
-    public InvalidDataFormatException(FunctionIdentifier fid, byte expectedTypeTag) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(),
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, byte expectedTypeTag) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag));
     }
 
-    public InvalidDataFormatException(FunctionIdentifier fid, String expectedType) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(), expectedType);
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, String expectedType) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), expectedType);
     }
 
-    public InvalidDataFormatException(FunctionIdentifier fid, Throwable cause, byte expectedTypeTag) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(), cause, expectedTypeTag);
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, Throwable cause,
+            byte expectedTypeTag) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), cause, expectedTypeTag);
         addSuppressed(cause);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
index 3690236..fde465d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
@@ -22,17 +22,11 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class OverflowException extends RuntimeDataException {
-
     // Overflow.
-    public OverflowException(FunctionIdentifier fid) {
-        super(ErrorCode.OVERFLOW, fid.getName());
+    public OverflowException(SourceLocation sourceLoc, FunctionIdentifier fid) {
+        super(ErrorCode.OVERFLOW, sourceLoc, fid.getName());
     }
-
-    // Overflow.
-    public OverflowException(String functionName) {
-        super(ErrorCode.OVERFLOW, functionName);
-    }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
index f6cf1c7..9fe602b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeMismatchException extends RuntimeDataException {
 
@@ -36,9 +37,24 @@
     }
 
     // Parameter type mistmatch.
+    public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, byte actualTypeTag,
+            byte... expectedTypeTags) {
+        super(ErrorCode.TYPE_MISMATCH, sourceLoc, fid.getName(), indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Parameter type mistmatch.
     public TypeMismatchException(String functionName, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
         super(ErrorCode.TYPE_MISMATCH, functionName, indexToPosition(i), toExpectedTypeString(expectedTypeTags),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
 
+    // Parameter type mistmatch.
+    public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, byte actualTypeTag,
+            byte... expectedTypeTags) {
+        super(ErrorCode.TYPE_MISMATCH, sourceLoc, functionName, indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
index 6471cbd..27bbcaa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
@@ -22,17 +22,12 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnderflowException extends RuntimeDataException {
 
     // Underflow.
-    public UnderflowException(FunctionIdentifier fid) {
-        super(ErrorCode.UNDERFLOW, fid.getName());
+    public UnderflowException(SourceLocation sourceLoc, FunctionIdentifier fid) {
+        super(ErrorCode.UNDERFLOW, sourceLoc, fid.getName());
     }
-
-    // Underflow.
-    public UnderflowException(String functionName) {
-        super(ErrorCode.UNDERFLOW, functionName);
-    }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
index f634649..25cfe29 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
@@ -23,16 +23,19 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedItemTypeException extends RuntimeDataException {
 
     // Unsupported item type.
-    public UnsupportedItemTypeException(FunctionIdentifier fid, byte itemTypeTag) {
-        super(ErrorCode.TYPE_ITEM, fid.getName(), EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte itemTypeTag) {
+        super(ErrorCode.TYPE_ITEM, sourceLoc, fid.getName(),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
     }
 
     // Unsupported item type.
-    public UnsupportedItemTypeException(String functionName, byte itemTypeTag) {
-        super(ErrorCode.TYPE_ITEM, functionName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, String functionName, byte itemTypeTag) {
+        super(ErrorCode.TYPE_ITEM, sourceLoc, functionName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
index 29b3819..32e9239 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
@@ -23,12 +23,13 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedTypeException extends RuntimeDataException {
 
     // Unsupported input type.
-    public UnsupportedTypeException(FunctionIdentifier fid, byte actualTypeTag) {
-        super(ErrorCode.TYPE_UNSUPPORTED, fid.getName(),
+    public UnsupportedTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
 
@@ -36,4 +37,10 @@
     public UnsupportedTypeException(String funcName, byte actualTypeTag) {
         super(ErrorCode.TYPE_UNSUPPORTED, funcName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, String funcName, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, funcName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index e237319..8b25e06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -79,6 +79,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class NonTaggedDataFormat implements IDataFormat {
@@ -120,7 +121,7 @@
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn) throws AlgebricksException {
+            List<String> fldName, int recordColumn, SourceLocation sourceLoc) throws AlgebricksException {
         IScalarEvaluatorFactory recordEvalFactory = new ColumnAccessEvalFactory(recordColumn);
 
         if (fldName.size() == 1) {
@@ -141,6 +142,7 @@
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                     IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     return fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -157,6 +159,7 @@
                 IScalarEvaluatorFactory fldNameEvalFactory =
                         new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                 IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                fDesc.setSourceLocation(sourceLoc);
                 return fDesc.createEvaluatorFactory(
                         new IScalarEvaluatorFactory[] { recordEvalFactory, fldNameEvalFactory });
             }
@@ -164,6 +167,7 @@
 
         if (fldName.size() > 1) {
             IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             return fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
         }
@@ -174,10 +178,10 @@
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
-            throws AlgebricksException {
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR,
+            SourceLocation sourceLoc) throws AlgebricksException {
         IScalarEvaluatorFactory evalFactory =
-                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn);
+                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn, sourceLoc);
         int numOfFields = isPointMBR ? dimension : dimension * 2;
         IScalarEvaluatorFactory[] evalFactories =
                 new IScalarEvaluatorFactory[numOfFields + (filterFieldName == null ? 0 : 1)];
@@ -209,7 +213,7 @@
         }
         if (filterFieldName != null) {
             evalFactories[numOfFields] =
-                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn);
+                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn, sourceLoc);
         }
         return evalFactories;
     }
@@ -217,7 +221,8 @@
     @SuppressWarnings("unchecked")
     @Override
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException {
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName, SourceLocation sourceLoc)
+            throws AlgebricksException {
         String[] names = recType.getFieldNames();
         int n = names.length;
         if (fldName.size() > 1) {
@@ -237,6 +242,7 @@
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                     IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     IScalarEvaluatorFactory evalFactory = fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -246,6 +252,7 @@
                     ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
                             new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
                             new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    partitionFun.setSourceLocation(sourceLoc);
                     return new Triple<>(evalFactory, partitionFun, recType.getFieldTypes()[i]);
                 }
             }
@@ -261,6 +268,7 @@
                 throw new AlgebricksException(e);
             }
             IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             IScalarEvaluatorFactory evalFactory =
                     fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
@@ -269,6 +277,7 @@
             ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
                     new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
                     new MutableObject<>(new ConstantExpression(new AsterixConstantValue(as))));
+            partitionFun.setSourceLocation(sourceLoc);
             return new Triple<>(evalFactory, partitionFun, recType.getSubFieldType(fldName));
         }
         throw new AlgebricksException("Could not find field " + fldName + " in the schema.");
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index b05dbed..271a7f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -104,7 +104,12 @@
 import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanOrEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.LessThanDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.LessThanOrEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.MissingIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NanIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NegInfIfEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.NotEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NullIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.PosInfIfEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
@@ -180,9 +185,11 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericATanDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericAddDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDivDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericPowerDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericCosDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDegreesDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericExpDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
@@ -190,6 +197,7 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericLogDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRadiansDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
@@ -201,6 +209,8 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericTruncDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
@@ -230,6 +240,7 @@
 import org.apache.asterix.runtime.evaluators.functions.StringRepeatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithLimitDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringReverseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringSplitDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
@@ -270,7 +281,10 @@
 import org.apache.asterix.runtime.evaluators.functions.records.RecordMergeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordNamesDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordPairsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordRenameDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordUnwrapDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustDateTimeForTimeZoneDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustTimeForTimeZoneDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDuartionFromDateDescriptor;
@@ -460,6 +474,7 @@
         fc.add(CreateUUIDDescriptor.FACTORY);
         fc.add(UUIDDescriptor.FACTORY);
         fc.add(CreateQueryUIDDescriptor.FACTORY);
+        fc.add(RandomDescriptor.FACTORY);
         fc.add(CurrentDateDescriptor.FACTORY);
         fc.add(CurrentTimeDescriptor.FACTORY);
         fc.add(CurrentDateTimeDescriptor.FACTORY);
@@ -481,10 +496,11 @@
         fc.addGenerated(NumericUnaryMinusDescriptor.FACTORY);
         fc.addGenerated(NumericAddDescriptor.FACTORY);
         fc.addGenerated(NumericDivideDescriptor.FACTORY);
+        fc.addGenerated(NumericDivDescriptor.FACTORY);
         fc.addGenerated(NumericMultiplyDescriptor.FACTORY);
         fc.addGenerated(NumericSubDescriptor.FACTORY);
         fc.addGenerated(NumericModuloDescriptor.FACTORY);
-        fc.addGenerated(NumericCaretDescriptor.FACTORY);
+        fc.addGenerated(NumericPowerDescriptor.FACTORY);
         fc.addGenerated(NotDescriptor.FACTORY);
         fc.addGenerated(LenDescriptor.FACTORY);
         fc.addGenerated(NumericAbsDescriptor.FACTORY);
@@ -496,6 +512,8 @@
         fc.addGenerated(NumericACosDescriptor.FACTORY);
         fc.addGenerated(NumericASinDescriptor.FACTORY);
         fc.addGenerated(NumericATanDescriptor.FACTORY);
+        fc.addGenerated(NumericDegreesDescriptor.FACTORY);
+        fc.addGenerated(NumericRadiansDescriptor.FACTORY);
         fc.addGenerated(NumericCosDescriptor.FACTORY);
         fc.addGenerated(NumericSinDescriptor.FACTORY);
         fc.addGenerated(NumericTanDescriptor.FACTORY);
@@ -515,6 +533,13 @@
         fc.addGenerated(LessThanOrEqualsDescriptor.FACTORY);
         fc.addGenerated(NotEqualsDescriptor.FACTORY);
 
+        // If-Equals functions
+        fc.addGenerated(MissingIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NullIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NanIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(PosInfIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NegInfIfEqualsDescriptor.FACTORY);
+
         // Binary functions
         fc.addGenerated(BinaryLengthDescriptor.FACTORY);
         fc.addGenerated(ParseBinaryDescriptor.FACTORY);
@@ -561,6 +586,7 @@
         fc.addGenerated(StringRepeatDescriptor.FACTORY);
         fc.addGenerated(StringReplaceDescriptor.FACTORY);
         fc.addGenerated(StringReplaceWithLimitDescriptor.FACTORY);
+        fc.addGenerated(StringReverseDescriptor.FACTORY);
         fc.addGenerated(StringSplitDescriptor.FACTORY);
 
         // Constructors
@@ -621,6 +647,9 @@
         fc.addGenerated(RecordRemoveFieldsDescriptor.FACTORY);
         fc.addGenerated(RecordLengthDescriptor.FACTORY);
         fc.addGenerated(RecordNamesDescriptor.FACTORY);
+        fc.addGenerated(RecordRemoveDescriptor.FACTORY);
+        fc.addGenerated(RecordRenameDescriptor.FACTORY);
+        fc.addGenerated(RecordUnwrapDescriptor.FACTORY);
 
         // Spatial and temporal type accessors
         fc.addGenerated(TemporalYearAccessor.FACTORY);
@@ -714,6 +743,9 @@
         // Record function
         fc.addGenerated(RecordPairsDescriptor.FACTORY);
 
+        // Other functions
+        fc.addGenerated(RandomWithSeedDescriptor.FACTORY);
+
         ServiceLoader.load(IFunctionRegistrant.class).iterator().forEachRemaining(c -> c.register(fc));
         return fc;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
index 2415556..4130490 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -33,7 +36,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 
 public class LSMIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
@@ -71,29 +73,26 @@
     @Override
     protected void initializeBulkLoader() throws HyracksDataException {
         ILSMIndex targetIndex = (ILSMIndex) index;
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
         if (usage.equals(BulkLoadUsage.LOAD)) {
-            // for a loaded dataset, we use the default Id 0 which is guaranteed to be smaller
-            // than Ids of all memory components
-
-            // TODO handle component Id for datasets loaded multiple times
-            // TODO move this piece of code to io operation callback
-            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
-            ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
-            LSMComponentIdUtils.persist(LSMComponentId.DEFAULT_COMPONENT_ID, diskComponent.getMetadata());
+            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    parameters);
         } else {
             primaryIndexHelper.open();
             primaryIndex = (ILSMIndex) primaryIndexHelper.getIndexInstance();
             List<ILSMDiskComponent> primaryComponents = primaryIndex.getDiskComponents();
-            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
             if (!primaryComponents.isEmpty()) {
-                // TODO move this piece of code to io operation callback
-                // Ideally, this should be done in io operation callback when a bulk load operation is finished
-                // However, currently we don't have an extensible callback mechanism to support this
                 ILSMComponentId bulkloadId = LSMComponentIdUtils.union(primaryComponents.get(0).getId(),
                         primaryComponents.get(primaryComponents.size() - 1).getId());
-                ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
-                LSMComponentIdUtils.persist(bulkloadId, diskComponent.getMetadata());
+                parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, bulkloadId);
+            } else {
+                parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+                        LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
             }
+            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    parameters);
+
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
index 6d9ec47..1029d6f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
@@ -19,7 +19,10 @@
 package org.apache.asterix.runtime.operators;
 
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorNodePushable.DeletedTupleCounter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -32,9 +35,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 
 /**
+ * Note: only used with correlated merge policy
  * This operator node is used to bulk load incoming tuples (scanned from the primary index)
  * into multiple disk components of the secondary index.
  * Incoming tuple format:
@@ -182,14 +185,12 @@
 
     private void loadNewComponent(int componentPos) throws HyracksDataException {
         endCurrentComponent();
-
         int numTuples = getNumDeletedTuples(componentPos);
         ILSMDiskComponent primaryComponent = primaryIndex.getDiskComponents().get(componentPos);
-        componentBulkLoader =
-                (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples, false);
-        ILSMDiskComponent diskComponent = componentBulkLoader.getComponent();
-        // TODO move this piece of code to io operation callback
-        LSMComponentIdUtils.persist(primaryComponent.getId(), diskComponent.getMetadata());
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, primaryComponent.getId());
+        componentBulkLoader = (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples,
+                false, parameters);
     }
 
     private void addAntiMatterTuple(ITupleReference tuple) throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
index 9376d1b..ac7fc89 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
@@ -276,7 +276,7 @@
 
     private boolean equalPrimaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
         for (int i = numTagFields + numSecondaryKeys; i < numTagFields + numPrimaryKeys + numSecondaryKeys; i++) {
-            if (!equalField(tuple1, tuple2, i)) {
+            if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
                 return false;
             }
         }
@@ -285,16 +285,10 @@
 
     private boolean equalSecondaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
         for (int i = numTagFields; i < numTagFields + numSecondaryKeys; i++) {
-            if (!equalField(tuple1, tuple2, i)) {
+            if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
                 return false;
             }
         }
         return true;
     }
-
-    private boolean equalField(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
-        return LSMSecondaryUpsertOperatorNodePushable.equals(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx),
-                tuple1.getFieldLength(fIdx), tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx),
-                tuple2.getFieldLength(fIdx));
-    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index a22e5e7..b928131 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -55,7 +56,7 @@
 public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
 
     private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
-    private int numberOfFields;
+    private final int numberOfFields;
     private AbstractIndexModificationOperationCallback abstractModCallback;
 
     public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
@@ -74,31 +75,6 @@
         abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
     }
 
-    public static boolean equals(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
-        if (aLength != bLength) {
-            return false;
-        }
-        for (int i = 0; i < aLength; i++) {
-            if (a[aOffset + i] != b[bOffset + i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    public static boolean equalTuples(PermutingFrameTupleReference t1, PermutingFrameTupleReference t2, int numOfFields)
-            throws HyracksDataException {
-        byte[] t1Data = t1.getFieldData(0);
-        byte[] t2Data = t2.getFieldData(0);
-        for (int i = 0; i < numOfFields; i++) {
-            if (!equals(t1Data, t1.getFieldStart(i), t1.getFieldLength(i), t2Data, t2.getFieldStart(i),
-                    t2.getFieldLength(i))) {
-                return false;
-            }
-        }
-        return true;
-    }
-
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
         accessor.reset(buffer);
@@ -117,7 +93,7 @@
                 }
                 // At least, one is not null
                 // If they are equal, then we skip
-                if (equalTuples(tuple, prevValueTuple, numberOfFields)) {
+                if (TupleUtils.equalTuples(tuple, prevValueTuple, numberOfFields)) {
                     continue;
                 }
                 if (!isOldValueMissing) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
index d61e9a0..7e42d14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
@@ -22,6 +22,7 @@
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.context.DatasetInfo;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ILockManager;
@@ -82,7 +83,13 @@
                     // lock the dataset granule
                     lockManager.lock(datasetId, -1, LockMode.S, txnCtx);
                     // flush the dataset synchronously
-                    datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+                    DatasetInfo datasetInfo = datasetLifeCycleManager.getDatasetInfo(datasetId.getId());
+                    // TODO: Remove the isOpen check and let it fail if flush is requested for a dataset that is closed
+                    synchronized (datasetLifeCycleManager) {
+                        if (datasetInfo.isOpen()) {
+                            datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+                        }
+                    }
                 } catch (ACIDException e) {
                     throw HyracksDataException.create(e);
                 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index 3409e61..eaf0d9c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,16 +58,18 @@
 
     @Override
     public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new ScanCollectionUnnestingFunctionFactory(args[0]);
+        return new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
     }
 
     public static class ScanCollectionUnnestingFunctionFactory implements IUnnestingEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory listEvalFactory;
+        private final SourceLocation sourceLoc;
 
-        public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg) {
+        public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg, SourceLocation sourceLoc) {
             this.listEvalFactory = arg;
+            this.sourceLoc = sourceLoc;
         }
 
         @Override
@@ -91,7 +94,7 @@
                     }
                     if (typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                             && typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.SCAN_COLLECTION, 0, typeTag,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.SCAN_COLLECTION, 0, typeTag,
                                 ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                     }
                     listAccessor.reset(inputVal.getByteArray(), inputVal.getStartOffset());
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index dc2df67..ee0b669 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -169,8 +169,8 @@
               <url>https://raw.githubusercontent.com/mojohaus/appassembler/appassembler-2.0.0/LICENSE.txt</url>
             </override>
             <override>
-              <gav>io.netty:netty-all:4.1.6.Final</gav>
-              <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.16.Final/NOTICE.txt</noticeUrl>
+              <gav>io.netty:netty-all:4.1.25.Final</gav>
+              <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.25.Final/NOTICE.txt</noticeUrl>
             </override>
           </overrides>
           <licenses>
@@ -407,6 +407,7 @@
             <usedDependency>org.codehaus.mojo.appassembler:appassembler-booter</usedDependency>
             <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
             <usedDependency>org.apache.asterix:asterix-dashboard</usedDependency>
+            <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
           </usedDependencies>
         </configuration>
       </plugin>
@@ -665,5 +666,14 @@
       <artifactId>asterix-dashboard</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-geo</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
index f7e0e34..d1699dd 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
@@ -39,15 +39,62 @@
   export JAVA_HOME
 fi
 
-[ -z "$JAVA_HOME" ] && {
-  echo "JAVA_HOME not set"
-  exit 1
-}
-"$JAVA_HOME/bin/java" -version 2>&1 | grep -q '1\.[89]' || {
-  echo "JAVA_HOME must be at version 1.8 or later:"
-  "$JAVA_HOME/bin/java" -version
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ]; then
+              if [ -x "/usr/libexec/java_home" ]; then
+                  JAVA_HOME=`/usr/libexec/java_home`
+              else
+                  JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+              fi
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+
+export JAVA_VERSION=$(java -version 2>&1 | head -1 | awk '{ print $NF }' | tr -d '"')
+case $JAVA_VERSION in
+  1.8*|1.9*|10*|11*)
+    ;;
+  *)
+  echo JAVA_HOME must be at version 1.8 or later, but is: $JAVA_VERSION
   exit 2
-}
+esac
+
 DIRNAME=$(dirname "$0")
 [ $(echo $DIRNAME | wc -l) -ne 1 ] && {
   echo "Paths with spaces are not supported"
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
index dbce377..522fb7c 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
@@ -39,13 +39,69 @@
   export JAVA_HOME
 fi
 
-[ -z "$JAVA_HOME" ] && {
-  echo "JAVA_HOME not set"
-  exit 1
-}
-"$JAVA_HOME/bin/java" -version 2>&1 | grep -q '1\.[89]' || {
+while [ -n "$1" ]; do
+  case $1 in
+    -f|-force) force=1;;
+    -help|--help|-usage|--usage) usage; exit 0;;
+    *) echo "ERROR: unknown argument '$1'"; usage; exit 1;;
+  esac
+  shift
+done
+
+if [ -z "$JAVA_HOME" -a -x /usr/libexec/java_home ]; then
+  JAVA_HOME=$(/usr/libexec/java_home)
+  export JAVA_HOME
+fi
+
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ]; then
+              if [ -x "/usr/libexec/java_home" ]; then
+                  JAVA_HOME=`/usr/libexec/java_home`
+              else
+                  JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+              fi
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+"$JAVACMD" -version 2>&1 | grep -q '1\.[89]' || {
   echo "JAVA_HOME must be at version 1.8 or later:"
-  "$JAVA_HOME/bin/java" -version
+  "$JAVACMD" -version
   exit 2
 }
 DIRNAME=$(dirname "$0")
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
index 766402c..defd378 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
@@ -23,11 +23,8 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.StringWriter;
-import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.utils.Servlets;
@@ -36,7 +33,6 @@
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -46,6 +42,10 @@
 import org.junit.rules.TestRule;
 import org.junit.runners.MethodSorters;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
 public class SampleLocalClusterIT {
 
@@ -118,11 +118,13 @@
     @Test
     public void test1_sanityQuery() throws Exception {
         TestExecutor testExecutor = new TestExecutor();
-        InputStream resultStream = testExecutor.executeQuery("1+1", OutputFormat.ADM,
-                new URI("http", null, "127.0.0.1", 19002, Servlets.AQL_QUERY, null, null), Collections.emptyList());
-        StringWriter sw = new StringWriter();
-        IOUtils.copy(resultStream, sw);
-        Assert.assertEquals("2", sw.toString().trim());
+        InputStream resultStream = testExecutor.executeQueryService("1+1;",
+                testExecutor.getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.ADM);
+        final ObjectMapper objectMapper = new ObjectMapper();
+        final ObjectNode response = objectMapper.readValue(resultStream, ObjectNode.class);
+        final JsonNode result = response.get("results");
+        Assert.assertEquals(1, result.size());
+        Assert.assertEquals(2, result.get(0).asInt());
     }
 
     @Test
diff --git a/asterixdb/asterix-test-framework/pom.xml b/asterixdb/asterix-test-framework/pom.xml
index 9822b20..2025d5a 100644
--- a/asterixdb/asterix-test-framework/pom.xml
+++ b/asterixdb/asterix-test-framework/pom.xml
@@ -51,7 +51,33 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <configuration>
+          <usedDependencies combine.children="append">
+            <usedDependency>com.sun.xml.bind:jaxb-core</usedDependency>
+            <usedDependency>com.sun.xml.bind:jaxb-impl</usedDependency>
+            <usedDependency>com.sun.activation:javax.activation</usedDependency>
+          </usedDependencies>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 
+  <dependencies>
+    <dependency>
+      <groupId>com.sun.xml.bind</groupId>
+      <artifactId>jaxb-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.xml.bind</groupId>
+      <artifactId>jaxb-impl</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.activation</groupId>
+      <artifactId>javax.activation</artifactId>
+    </dependency>
+  </dependencies>
+
 </project>
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
index 7a34648..4735c67 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
@@ -202,6 +202,11 @@
         return filePath;
     }
 
+    public boolean isSourceLocationExpected(CompilationUnit cUnit) {
+        Boolean v = cUnit.isSourceLocation();
+        return v != null ? v : testSuite.isSourceLocation();
+    }
+
     @Override
     public String toString() {
         StringBuilder sb = new StringBuilder(testCase.getFilePath());
diff --git a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
index 7a5387c..7a4feea 100644
--- a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
+++ b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
@@ -63,6 +63,14 @@
             </xs:annotation>
          </xs:attribute>
 
+         <xs:attribute name="SourceLocation" type="xs:boolean" default="false">
+            <xs:annotation>
+               <xs:documentation>
+                  whether to expect a source location in error messages
+               </xs:documentation>
+            </xs:annotation>
+         </xs:attribute>
+
       </xs:complexType>
 
       <xs:unique name="unique-test-group">
@@ -160,6 +168,17 @@
                      </xs:annotation>
                   </xs:element>
 
+                  <!-- Whether the source location is expected in the error message -->
+
+                  <xs:element name="source-location" type="xs:boolean" minOccurs="0">
+                     <xs:annotation>
+                        <xs:documentation>
+                           Whether to expect a source location in the error message
+                           (default is defined by the test-suite element)
+                        </xs:documentation>
+                     </xs:annotation>
+                  </xs:element>
+
                </xs:sequence>
 
                <!-- This name is always equal to the name of the test case -->
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 07dcf5d..e6e522e5 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -163,5 +163,9 @@
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
+    <dependency>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 0375c30..e1963cb 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -186,7 +186,7 @@
         }
 
         resourceCache.put(resource.getPath(), resource);
-        indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(0);
+        indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(null, 0);
         //if replication enabled, send resource metadata info to remote nodes
         if (isReplicationEnabled) {
             createReplicationJob(ReplicationOperation.REPLICATE, resourceFile);
@@ -429,15 +429,20 @@
     }
 
     private void deleteIndexInvalidComponents(File index) throws IOException, ParseException {
+        final Format formatter = THREAD_LOCAL_FORMATTER.get();
+        final File[] indexComponentFiles = index.listFiles(COMPONENT_FILES_FILTER);
+        if (indexComponentFiles == null) {
+            throw new IOException(index + " doesn't exist or an IO error occurred");
+        }
         final Optional<String> validComponentTimestamp = getIndexCheckpointManager(index).getValidComponentTimestamp();
         if (!validComponentTimestamp.isPresent()) {
-            // index doesn't have any components
-            return;
-        }
-        final Format formatter = THREAD_LOCAL_FORMATTER.get();
-        final Date validTimestamp = (Date) formatter.parseObject(validComponentTimestamp.get());
-        final File[] indexComponentFiles = index.listFiles(COMPONENT_FILES_FILTER);
-        if (indexComponentFiles != null) {
+            // index doesn't have any valid component, delete all
+            for (File componentFile : indexComponentFiles) {
+                LOGGER.info(() -> "Deleting invalid component file: " + componentFile.getAbsolutePath());
+                Files.delete(componentFile.toPath());
+            }
+        } else {
+            final Date validTimestamp = (Date) formatter.parseObject(validComponentTimestamp.get());
             for (File componentFile : indexComponentFiles) {
                 // delete any file with startTime > validTimestamp
                 final String fileStartTimeStr =
@@ -505,7 +510,8 @@
      * e.g. a component file 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439_b
      * will return a component id 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439
      *
-     * @param componentFile any component file
+     * @param componentFile
+     *            any component file
      * @return The component id
      */
     public static String getComponentId(String componentFile) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
index bbfde38..708e8dc 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
@@ -21,12 +21,12 @@
 
 import org.apache.asterix.common.api.IJobEventListenerFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 
-public class CommitRuntimeFactory implements IPushRuntimeFactory {
+public class CommitRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index c91d233..b26342e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.common.exceptions.ACIDException;
@@ -36,30 +35,37 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMaps;
+import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
 /**
  * A concurrent implementation of the ILockManager interface.
  *
  * @see ResourceGroupTable
  * @see ResourceGroup
  */
+@SuppressWarnings("squid:RedundantThrowsDeclarationCheck") // throws ACIDException
 public class ConcurrentLockManager implements ILockManager, ILifeCycleComponent {
 
     static final Logger LOGGER = LogManager.getLogger();
     static final Level LVL = Level.TRACE;
-    public static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
+    private static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
 
-    public static final int NIL = -1;
-    public static final long NILL = -1L;
+    private static final int NIL = -1;
+    private static final long NILL = -1L;
 
-    public static final boolean DEBUG_MODE = false;//true
-    public static final boolean CHECK_CONSISTENCY = false;
+    private static final boolean DEBUG_MODE = false;//true
+    private static final boolean CHECK_CONSISTENCY = false;
 
-    private ResourceGroupTable table;
-    private ResourceArenaManager resArenaMgr;
-    private RequestArenaManager reqArenaMgr;
-    private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Long, Long> txnId2TxnSlotMap;
-    private LockManagerStats stats = new LockManagerStats(10000);
+    private final ResourceGroupTable table;
+    private final ResourceArenaManager resArenaMgr;
+    private final RequestArenaManager reqArenaMgr;
+    private final JobArenaManager jobArenaMgr;
+    private final Long2LongMap txnId2TxnSlotMap;
+    private final LockManagerStats stats = new LockManagerStats(10000);
 
     enum LockAction {
         ERR(false, false),
@@ -77,7 +83,7 @@
         }
     }
 
-    static LockAction[][] ACTION_MATRIX = {
+    private static final LockAction[][] ACTION_MATRIX = {
             // new    NL              IS               IX                S                X
             { LockAction.ERR, LockAction.UPD, LockAction.UPD, LockAction.UPD, LockAction.UPD }, // NL
             { LockAction.ERR, LockAction.GET, LockAction.UPD, LockAction.UPD, LockAction.WAIT }, // IS
@@ -97,7 +103,7 @@
         resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
         reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
         jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
-        txnId2TxnSlotMap = new ConcurrentHashMap<>();
+        txnId2TxnSlotMap = Long2LongMaps.synchronize(new Long2LongOpenHashMap());
     }
 
     @Override
@@ -129,10 +135,10 @@
                             enqueueWaiter(group, reqSlot, resSlot, jobSlot, act, txnContext);
                             break;
                         }
-                        //no break
+                        //fall-through
                     case UPD:
                         resArenaMgr.setMaxMode(resSlot, lockMode);
-                        // no break
+                        //fall-through
                     case GET:
                         addHolder(reqSlot, resSlot, jobSlot);
                         locked = true;
@@ -185,31 +191,35 @@
         void pop();
     }
 
-    static class NOPTracker implements DeadlockTracker {
+    private static class NOPTracker implements DeadlockTracker {
         static final DeadlockTracker INSTANCE = new NOPTracker();
 
         @Override
         public void pushResource(long resSlot) {
+            // no-op
         }
 
         @Override
         public void pushRequest(long reqSlot) {
+            // no-op
         }
 
         @Override
         public void pushJob(long jobSlot) {
+            // no-op
         }
 
         @Override
         public void pop() {
+            // no-op
         }
     }
 
-    static class CollectingTracker implements DeadlockTracker {
+    private static class CollectingTracker implements DeadlockTracker {
 
         static final boolean DEBUG = false;
 
-        ArrayList<Long> slots = new ArrayList<>();
+        LongList slots = new LongArrayList();
         ArrayList<String> types = new ArrayList<>();
 
         @Override
@@ -217,7 +227,7 @@
             types.add("Resource");
             slots.add(resSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
@@ -226,7 +236,7 @@
             types.add("Request");
             slots.add(reqSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
@@ -235,24 +245,24 @@
             types.add("Job");
             slots.add(jobSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
         @Override
         public void pop() {
             if (DEBUG) {
-                System.err.println("pop " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("pop " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
             types.remove(types.size() - 1);
-            slots.remove(slots.size() - 1);
+            slots.removeLong(slots.size() - 1);
         }
 
         @Override
         public String toString() {
             StringBuilder sb = new StringBuilder();
             for (int i = 0; i < slots.size(); ++i) {
-                sb.append(types.get(i) + " " + TypeUtil.Global.toString(slots.get(i)) + "\n");
+                sb.append(types.get(i)).append(" ").append(TypeUtil.Global.toString(slots.getLong(i))).append("\n");
             }
             return sb.toString();
         }
@@ -270,15 +280,11 @@
      * @return true if a cycle would be introduced, false otherwise
      */
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
-        if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
-            /**
-             * Due to the deadlock-free locking protocol, deadlock is not possible.
-             * So, this method always returns false.
-             */
-            return false;
-        } else {
-            return introducesDeadlock(resSlot, jobSlot, tracker, 0);
-        }
+        /*
+         * Due to the deadlock-free locking protocol, deadlock is not possible.
+         * So, this method always returns false in that case
+         */
+        return !ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL && introducesDeadlock(resSlot, jobSlot, tracker, 0);
     }
 
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker,
@@ -298,20 +304,20 @@
                 // The scanWaiters flag indicates if we are currently scanning the waiters (true) or the upgraders
                 // (false).
                 boolean scanWaiters = true;
-                long waiter = jobArenaMgr.getLastWaiter(holderJobSlot);
-                if (waiter < 0 && scanWaiters) {
+                long jobWaiter = jobArenaMgr.getLastWaiter(holderJobSlot);
+                if (jobWaiter < 0) {
                     scanWaiters = false;
-                    waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+                    jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
                 }
-                while (waiter >= 0) {
-                    long waitingOnResSlot = reqArenaMgr.getResourceId(waiter);
+                while (jobWaiter >= 0) {
+                    long waitingOnResSlot = reqArenaMgr.getResourceId(jobWaiter);
                     if (introducesDeadlock(waitingOnResSlot, jobSlot, tracker, depth + 1)) {
                         return true;
                     }
-                    waiter = reqArenaMgr.getNextJobRequest(waiter);
-                    if (waiter < 0 && scanWaiters) {
+                    jobWaiter = reqArenaMgr.getNextJobRequest(jobWaiter);
+                    if (jobWaiter < 0 && scanWaiters) {
                         scanWaiters = false;
-                        waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+                        jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
                     }
                 }
 
@@ -407,7 +413,7 @@
             switch (act) {
                 case UPD:
                     resArenaMgr.setMaxMode(resSlot, lockMode);
-                    // no break
+                    //fall-through
                 case GET:
                     addHolder(reqSlot, resSlot, jobSlot);
                     return true;
@@ -532,8 +538,8 @@
         stats.releaseLocks();
 
         long txnId = txnContext.getTxnId().getId();
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
             // we don't know the job, so there are no locks for it - we're done
             return;
         }
@@ -565,15 +571,15 @@
     }
 
     private long findOrAllocJobSlot(long txnId) {
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
-            jobSlot = new Long(jobArenaMgr.allocate());
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
+            jobSlot = jobArenaMgr.allocate();
             if (DEBUG_MODE) {
                 LOGGER.trace("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + txnId + ")");
             }
             jobArenaMgr.setTxnId(jobSlot, txnId);
-            Long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
-            if (oldSlot != null) {
+            long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
+            if (oldSlot != 0) {
                 // if another thread allocated a slot for this jobThreadId between
                 // get(..) and putIfAbsent(..), we'll use that slot and
                 // deallocate the one we allocated
@@ -584,7 +590,7 @@
                 jobSlot = oldSlot;
             }
         }
-        assert (jobSlot >= 0);
+        assert jobSlot > 0;
         return jobSlot;
     }
 
@@ -754,7 +760,7 @@
         void remove(long request, long resource, long job);
     }
 
-    final Queue waiter = new Queue() {
+    private final Queue waiter = new Queue() {
         @Override
         public void add(long request, long resource, long job) {
             long waiter = resArenaMgr.getFirstWaiter(resource);
@@ -788,7 +794,7 @@
         }
     };
 
-    final Queue upgrader = new Queue() {
+    private final Queue upgrader = new Queue() {
         @Override
         public void add(long request, long resource, long job) {
             long upgrader = resArenaMgr.getFirstUpgrader(resource);
@@ -987,6 +993,7 @@
                 }
             }
         } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
             throw new IllegalStateException("interrupted", e);
         }
     }
@@ -1015,8 +1022,8 @@
      * @return the slot of the request, if the lock request is found, NILL otherwise
      */
     private long findLockInJobQueue(final int dsId, final int entityHashValue, final long txnId, byte lockMode) {
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
             return NILL;
         }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
index 26261c2..6a5372e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
@@ -19,17 +19,17 @@
 
 package org.apache.asterix.transaction.management.service.locking;
 
-import java.util.concurrent.ConcurrentHashMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
 
 public class DumpTablePrinter implements TablePrinter {
     private ResourceGroupTable table;
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Long, Long> txnIdToJobSlotMap;
+    private Long2LongMap txnIdToJobSlotMap;
 
     DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
-            JobArenaManager jobArenaMgr, ConcurrentHashMap<Long, Long> txnIdToJobSlotMap) {
+            JobArenaManager jobArenaMgr, Long2LongMap txnIdToJobSlotMap) {
         this.table = table;
         this.resArenaMgr = resArenaMgr;
         this.reqArenaMgr = reqArenaMgr;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
deleted file mode 100644
index 5bf5ad6..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
+++ /dev/null
@@ -1,595 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-
-/**
- * PrimitiveIntHashMap supports primitive int type as key and value.
- * The hash map grows when the available slots in a bucket are overflowed.
- * Also, the hash map shrinks according to the following shrink policy.
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class PrimitiveIntHashMap {
-    private final int CHILD_BUCKETS; //INIT_NUM_OF_BUCKETS;
-    private final int NUM_OF_SLOTS; //NUM_OF_SLOTS_IN_A_BUCKET;
-    private final int SHRINK_TIMER_THRESHOLD;
-
-    private int occupiedSlots;
-    private ArrayList<ChildIntArrayManager> pArray; //parent array
-    private int hashMod;
-    private long shrinkTimer;
-    private boolean isShrinkTimerOn;
-    private int iterBucketIndex;
-    private int iterSlotIndex;
-    private int iterChildIndex;
-    private KeyValuePair iterPair;
-
-    //    ////////////////////////////////////////////////
-    //    // begin of unit test
-    //    ////////////////////////////////////////////////
-    //
-    //    /**
-    //     * @param args
-    //     */
-    //    public static void main(String[] args) {
-    //        int i, j;
-    //        int k = 0;
-    //        int num = 5;
-    //        int key[] = new int[500];
-    //        int val[] = new int[500];
-    //        KeyValuePair pair;
-    //        PrimitiveIntHashMap map = new PrimitiveIntHashMap(1<<4, 1<<3, 5);
-    //
-    //        for (j=0; j < num; j++) {
-    //
-    //            k += 100;
-    //            //generate data
-    //            for (i=0; i < k; i++) {
-    //                key[i] = i;
-    //                val[i] = i;
-    //            }
-    //
-    //            //put data to map
-    //            for (i=0; i < k-30; i++) {
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            //put data to map
-    //            for (i=0; i < k-30; i++) {
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            map.beginIterate();
-    //            pair = map.getNextKeyValue();
-    //            i = 0;
-    //            while (pair != null) {
-    //                i++;
-    //                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //                pair = map.getNextKeyValue();
-    //            }
-    //
-    //            //System.out.println(map.prettyPrint());
-    //
-    //            for (i=k-20; i< k; i++) { //skip X70~X79
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            System.out.println(map.prettyPrint());
-    //
-    //            //remove data to map
-    //            for (i=0; i < k-10; i++) {
-    //                map.remove(key[i]);
-    //                try {
-    //                    Thread.currentThread().sleep(1);
-    //                } catch (InterruptedException e) {
-    //                    e.printStackTrace();
-    //                }
-    //            }
-    //
-    //            map.beginIterate();
-    //            pair = map.getNextKeyValue();
-    //            i = 0;
-    //            while (pair != null) {
-    //                i++;
-    //                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //                pair = map.getNextKeyValue();
-    //            }
-    //
-    //            //remove data to map
-    //            for (i=0; i < k-10; i++) {
-    //                map.remove(key[i]);
-    //                try {
-    //                    Thread.currentThread().sleep(1);
-    //                } catch (InterruptedException e) {
-    //                    // TODO Auto-generated catch block
-    //                    e.printStackTrace();
-    //                }
-    //            }
-    //
-    //            System.out.println(map.prettyPrint());
-    //
-    //            //get data from map
-    //            for (i=0; i < k; i++) {
-    //                System.out.println(""+i+"=> key:"+ key[i] + ", val:"+val[i] +", result: " + map.get(key[i]));
-    //            }
-    //        }
-    //
-    //        map.beginIterate();
-    //        pair = map.getNextKeyValue();
-    //        i = 0;
-    //        while (pair != null) {
-    //            i++;
-    //            System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //            pair = map.getNextKeyValue();
-    //        }
-    //    }
-    //
-    //    ////////////////////////////////////////////////
-    //    // end of unit test
-    //    ////////////////////////////////////////////////
-
-    public PrimitiveIntHashMap() {
-        CHILD_BUCKETS = 1 << 9; //INIT_NUM_OF_BUCKETS;
-        NUM_OF_SLOTS = 1 << 3; //NUM_OF_SLOTS_IN_A_BUCKET;
-        SHRINK_TIMER_THRESHOLD = 120000; //2min
-        pArray = new ArrayList<ChildIntArrayManager>();
-        pArray.add(new ChildIntArrayManager(this));
-        hashMod = CHILD_BUCKETS;
-        occupiedSlots = 0;
-        iterPair = new KeyValuePair();
-    }
-
-    public PrimitiveIntHashMap(int childBuckets, int numOfSlots, int shrinkTimerThreshold) {
-        CHILD_BUCKETS = childBuckets;
-        NUM_OF_SLOTS = numOfSlots;
-        SHRINK_TIMER_THRESHOLD = shrinkTimerThreshold;
-        pArray = new ArrayList<ChildIntArrayManager>();
-        pArray.add(new ChildIntArrayManager(this));
-        hashMod = CHILD_BUCKETS;
-        occupiedSlots = 0;
-        iterPair = new KeyValuePair();
-    }
-
-    public void put(int key, int value) {
-        int growCount = 0;
-        int bucketNum = hash(key);
-        ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
-        while (child.isFull(bucketNum % CHILD_BUCKETS)) {
-            growHashMap();
-            bucketNum = hash(key);
-            child = pArray.get(bucketNum / CHILD_BUCKETS);
-            if (growCount > 2) {
-                //changeHashFunc();
-            }
-            growCount++;
-        }
-        occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, false);
-    }
-
-    public void upsert(int key, int value) {
-        int growCount = 0;
-        int bucketNum = hash(key);
-        ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
-        while (child.isFull(bucketNum % CHILD_BUCKETS)) {
-            growHashMap();
-            bucketNum = hash(key);
-            child = pArray.get(bucketNum / CHILD_BUCKETS);
-            if (growCount > 2) {
-                //changeHashFunc();
-            }
-            growCount++;
-        }
-        occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, true);
-    }
-
-    private int hash(int key) {
-        return key % hashMod;
-    }
-
-    private void growHashMap() {
-        int size = pArray.size();
-        int i;
-
-        //grow buckets by adding more child
-        for (i = 0; i < size; i++) {
-            pArray.add(new ChildIntArrayManager(this));
-        }
-
-        //increase hashMod
-        hashMod *= 2;
-
-        //re-hash
-        rehash(0, size, hashMod / 2);
-    }
-
-    private void shrinkHashMap() {
-        int size = pArray.size();
-        int i;
-
-        //decrease hashMod
-        hashMod /= 2;
-
-        //re-hash
-        rehash(size / 2, size, hashMod * 2);
-
-        //shrink buckets by removing child(s)
-        for (i = size - 1; i >= size / 2; i--) {
-            pArray.remove(i);
-        }
-    }
-
-    private void rehash(int begin, int end, int oldHashMod) {
-        int i, j, k;
-        int key, value;
-        ChildIntArrayManager child;
-
-        //re-hash
-        for (i = begin; i < end; i++) {
-            child = pArray.get(i);
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                if (child.cArray[j][0] == 0) {
-                    continue;
-                }
-                for (k = 1; k < NUM_OF_SLOTS; k++) {
-                    //if the hashValue of the key is different, then re-hash it.
-                    key = child.cArray[j][k * 2];
-                    if (hash(key) != key % oldHashMod) {
-                        value = child.cArray[j][k * 2 + 1];
-                        //remove existing key and value
-                        //Notice! To avoid bucket iteration, child.remove() is not used.
-                        child.cArray[j][k * 2] = -1;
-                        child.cArray[j][0]--;
-                        //re-hash it
-                        pArray.get(hash(key) / CHILD_BUCKETS).put(hash(key) % CHILD_BUCKETS, key, value, false);
-                    }
-                }
-            }
-        }
-    }
-
-    //    private void changeHashFunc() {
-    //        //TODO need to implement.
-    //        throw new UnsupportedOperationException("changeHashFunc() not implemented");
-    //    }
-
-    public int get(int key) {
-        int bucketNum = hash(key);
-        return pArray.get(bucketNum / CHILD_BUCKETS).get(bucketNum % CHILD_BUCKETS, key);
-    }
-
-    public void remove(int key) {
-        int bucketNum = hash(key);
-        occupiedSlots -= pArray.get(bucketNum / CHILD_BUCKETS).remove(bucketNum % CHILD_BUCKETS, key);
-
-        if (needShrink()) {
-            shrinkHashMap();
-        }
-    }
-
-    /**
-     * Shrink policy:
-     * Shrink when the resource under-utilization lasts for a certain amount of time.
-     *
-     * @return
-     */
-    private boolean needShrink() {
-        int size = pArray.size();
-        int usedSlots = occupiedSlots;
-        if (usedSlots == 0) {
-            usedSlots = 1;
-        }
-        if (size > 1 && size * CHILD_BUCKETS * NUM_OF_SLOTS / usedSlots >= 3 && isSafeToShrink()) {
-            if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
-                    isShrinkTimerOn = false;
-                    return true;
-                }
-            } else {
-                //turn on timer
-                isShrinkTimerOn = true;
-                shrinkTimer = System.currentTimeMillis();
-            }
-        } else {
-            //turn off timer
-            isShrinkTimerOn = false;
-        }
-        return false;
-    }
-
-    private boolean isSafeToShrink() {
-        int i, j;
-        int size = pArray.size();
-        //Child: 0, 1, 2, 3, 4, 5, 6, 7
-        //[HChild(Head Child):0 and TChild(Tail Child): 4], [1(H),5(T)], [2(H),6(T)] and so on.
-        //When the map shrinks, the sum of occupied slots in H/TChild should not exceed the NUM_OF_SLOTS-1.
-        //Then it is safe to shrink. Otherwise, unsafe.
-        ChildIntArrayManager HChild, TChild;
-
-        for (i = 0; i < size / 2; i++) {
-            HChild = pArray.get(i);
-            TChild = pArray.get(size / 2 + i);
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                if (HChild.cArray[j][0] + TChild.cArray[j][0] > NUM_OF_SLOTS - 1) {
-                    return false;
-                }
-            }
-        }
-        return true;
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder("\n########### PrimitiveIntHashMap Status #############\n");
-        ChildIntArrayManager child;
-        int i, j, k;
-        int size = pArray.size();
-        for (i = 0; i < size; i++) {
-            child = pArray.get(i);
-            s.append("child[").append(i).append("]\n");
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                s.append(j).append(" ");
-                for (k = 0; k < NUM_OF_SLOTS; k++) {
-                    s.append("[").append(child.cArray[j][k * 2]).append(",").append(child.cArray[j][k * 2 + 1])
-                            .append("] ");
-                }
-                s.append("\n");
-            }
-        }
-        return s.toString();
-    }
-
-    public int getNumOfSlots() {
-        return NUM_OF_SLOTS;
-    }
-
-    public int getNumOfChildBuckets() {
-        return CHILD_BUCKETS;
-    }
-
-    public void clear(boolean needShrink) {
-        int size = pArray.size();
-        for (int i = size - 1; i >= 0; i--) {
-            if (needShrink && i != 0) {
-                pArray.remove(i);
-            } else {
-                pArray.get(i).clear();
-            }
-        }
-        occupiedSlots = 0;
-    }
-
-    ///////////////////////////////////////
-    // iterate method
-    ///////////////////////////////////////
-
-    public void beginIterate() {
-        iterChildIndex = 0;
-        iterBucketIndex = 0;
-        iterSlotIndex = 1;
-    }
-
-    public KeyValuePair getNextKeyValue() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
-                    iterSlotIndex++;
-                    return iterPair;
-                }
-            }
-        }
-        return null;
-    }
-
-    public int getNextKey() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterSlotIndex++;
-                    return iterPair.key;
-                }
-            }
-        }
-        return -1;
-    }
-
-    public int getNextValue() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
-                    iterSlotIndex++;
-                    return iterPair.value;
-                }
-            }
-        }
-        return -1;
-    }
-
-    public static class KeyValuePair {
-        public int key;
-        public int value;
-    }
-}
-
-class ChildIntArrayManager {
-    private final int DIM1_SIZE;
-    private final int DIM2_SIZE;
-    private final int NUM_OF_SLOTS;
-    public int[][] cArray; //child array
-
-    public ChildIntArrayManager(PrimitiveIntHashMap parentHashMap) {
-        DIM1_SIZE = parentHashMap.getNumOfChildBuckets();
-        DIM2_SIZE = parentHashMap.getNumOfSlots() * 2; //2: Array of [key, value] pair
-        NUM_OF_SLOTS = parentHashMap.getNumOfSlots();
-        initialize();
-    }
-
-    private void initialize() {
-        cArray = new int[DIM1_SIZE][DIM2_SIZE];
-        int i, j;
-        for (i = 0; i < DIM1_SIZE; i++) {
-            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
-            //cArray[i][1] is not used.
-            cArray[i][0] = 0;
-            for (j = 1; j < NUM_OF_SLOTS; j++) {
-                cArray[i][j * 2] = -1; // -1 represent that the slot is empty
-            }
-        }
-    }
-
-    public void clear() {
-        int i, j;
-        for (i = 0; i < DIM1_SIZE; i++) {
-            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
-            //cArray[i][1] is not used.
-            if (cArray[i][0] == 0) {
-                continue;
-            }
-            cArray[i][0] = 0;
-            for (j = 1; j < NUM_OF_SLOTS; j++) {
-                cArray[i][j * 2] = -1; // -1 represent that the slot is empty
-            }
-        }
-    }
-
-    public void deinitialize() {
-        cArray = null;
-    }
-
-    public void allocate() {
-        initialize();
-    }
-
-    public boolean isFull(int bucketNum) {
-        return cArray[bucketNum][0] == NUM_OF_SLOTS - 1;
-    }
-
-    public boolean isEmpty(int bucketNum) {
-        return cArray[bucketNum][0] == 0;
-    }
-
-    /**
-     * Put key,value into a slot in the bucket if the key doesn't exist.
-     * Update value if the key exists and if isUpsert is true
-     * No need to call get() to check the existence of the key before calling put().
-     * Notice! Caller should make sure that there is an available slot.
-     *
-     * @param bucketNum
-     * @param key
-     * @param value
-     * @param isUpsert
-     * @return 1 for new insertion, 0 for key duplication
-     */
-    public int put(int bucketNum, int key, int value, boolean isUpsert) {
-        int i;
-        int emptySlot = -1;
-
-        if (cArray[bucketNum][0] == 0) {
-            cArray[bucketNum][2] = key;
-            cArray[bucketNum][3] = value;
-            cArray[bucketNum][0]++;
-            return 1;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                if (isUpsert) {
-                    cArray[bucketNum][i * 2 + 1] = value;
-                }
-                return 0;
-            } else if (cArray[bucketNum][i * 2] == -1) {
-                emptySlot = i;
-            }
-        }
-
-        if (emptySlot == -1) {
-            throw new UnsupportedOperationException("error");
-        }
-
-        cArray[bucketNum][emptySlot * 2] = key;
-        cArray[bucketNum][emptySlot * 2 + 1] = value;
-        cArray[bucketNum][0]++;
-        return 1;
-    }
-
-    public int get(int bucketNum, int key) {
-        int i;
-
-        if (cArray[bucketNum][0] == 0) {
-            return -1;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                return cArray[bucketNum][i * 2 + 1];
-            }
-        }
-        return -1;
-    }
-
-    /**
-     * remove key if it exists. Otherwise, ignore it.
-     *
-     * @param bucketNum
-     * @param key
-     * @return 1 for success, 0 if the key doesn't exist
-     */
-    public int remove(int bucketNum, int key) {
-        int i;
-
-        if (cArray[bucketNum][0] == 0) {
-            return 0;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                cArray[bucketNum][i * 2] = -1;
-                cArray[bucketNum][0]--;
-                return 1;
-            }
-        }
-
-        return 0;
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 21268e5..4085fb4 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -89,8 +89,7 @@
     public void append(ILogRecord logRecord, long appendLsn) {
         logRecord.writeLogRecord(appendBuffer);
 
-        if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
-                && logRecord.getLogType() != LogType.WAIT) {
+        if (isLocalTransactionLog(logRecord)) {
             logRecord.getTxnCtx().setLastLSN(appendLsn);
         }
 
@@ -100,13 +99,10 @@
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
             if (logRecord.getLogSource() == LogSource.LOCAL) {
-                if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
-                        || logRecord.getLogType() == LogType.WAIT) {
+                if (syncPendingNonFlushLog(logRecord)) {
                     logRecord.isFlushed(false);
                     syncCommitQ.add(logRecord);
-                }
-                if (logRecord.getLogType() == LogType.FLUSH) {
-                    logRecord.isFlushed(false);
+                } else if (logRecord.getLogType() == LogType.FLUSH) {
                     flushQ.add(logRecord);
                 }
             } else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
@@ -117,6 +113,16 @@
         }
     }
 
+    private boolean syncPendingNonFlushLog(ILogRecord logRecord) {
+        return logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                || logRecord.getLogType() == LogType.WAIT || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES;
+    }
+
+    private boolean isLocalTransactionLog(ILogRecord logRecord) {
+        return logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
+                && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
+    }
+
     @Override
     public void setFileChannel(FileChannel fileChannel) {
         this.fileChannel = fileChannel;
@@ -231,7 +237,8 @@
                         notifyJobTermination();
                     } else if (logRecord.getLogType() == LogType.FLUSH) {
                         notifyFlushTermination();
-                    } else if (logRecord.getLogType() == LogType.WAIT) {
+                    } else if (logRecord.getLogType() == LogType.WAIT
+                            || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
                         notifyWaitTermination();
                     }
                 } else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index 736de07..be227ec 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -32,7 +32,6 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -134,13 +133,33 @@
 
     @Override
     public void log(ILogRecord logRecord) {
-        if (logRecord.getLogType() == LogType.FLUSH) {
-            flushLogsQ.add(logRecord);
-            return;
+        if (!logToFlushQueue(logRecord)) {
+            appendToLogTail(logRecord);
         }
-        appendToLogTail(logRecord);
     }
 
+    @SuppressWarnings("squid:S2445")
+    protected boolean logToFlushQueue(ILogRecord logRecord) {
+        //Remote flush logs do not need to be flushed separately since they may not trigger local flush
+        if ((logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL)
+                || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+            logRecord.isFlushed(false);
+            flushLogsQ.add(logRecord);
+            if (logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+                InvokeUtil.doUninterruptibly(() -> {
+                    synchronized (logRecord) {
+                        while (!logRecord.isFlushed()) {
+                            logRecord.wait();
+                        }
+                    }
+                });
+            }
+            return true;
+        }
+        return false;
+    }
+
+    @SuppressWarnings("squid:S2445")
     protected void appendToLogTail(ILogRecord logRecord) {
         syncAppendToLogTail(logRecord);
         if (waitForFlush(logRecord) && !logRecord.isFlushed()) {
@@ -161,7 +180,7 @@
 
     synchronized void syncAppendToLogTail(ILogRecord logRecord) {
         if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
-                && logRecord.getLogType() != LogType.WAIT) {
+                && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES) {
             ITransactionContext txnCtx = logRecord.getTxnCtx();
             if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
                 throw new ACIDException(
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 1e13883..d2e9629 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -40,9 +40,11 @@
         super(txnSubsystem);
     }
 
+    @SuppressWarnings("squid:S2445")
     @Override
     public void log(ILogRecord logRecord) {
-        boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT;
+        boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT
+                && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
         if (shouldReplicate) {
             switch (logRecord.getLogType()) {
                 case LogType.ENTITY_COMMIT:
@@ -63,16 +65,12 @@
             }
         }
         logRecord.setReplicate(shouldReplicate);
-
-        //Remote flush logs do not need to be flushed separately since they may not trigger local flush
-        if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
-            flushLogsQ.add(logRecord);
-            return;
+        if (!logToFlushQueue(logRecord)) {
+            appendToLogTail(logRecord);
         }
-
-        appendToLogTail(logRecord);
     }
 
+    @SuppressWarnings("squid:S2445")
     @Override
     protected void appendToLogTail(ILogRecord logRecord) {
         syncAppendToLogTail(logRecord);
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index a56ffd8..3885894 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -475,12 +475,12 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.20.1</version>
+          <version>2.21.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-failsafe-plugin</artifactId>
-          <version>2.20.1</version>
+          <version>2.21.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.hyracks</groupId>
@@ -528,7 +528,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-javadoc-plugin</artifactId>
-          <version>3.0.0-M1</version>
+          <version>3.0.1</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -543,7 +543,7 @@
         <plugin>
           <groupId>org.jvnet.jaxb2.maven2</groupId>
           <artifactId>maven-jaxb2-plugin</artifactId>
-          <version>0.13.2</version>
+          <version>0.14.0</version>
         </plugin>
         <plugin>
           <groupId>pl.project13.maven</groupId>
@@ -690,15 +690,6 @@
       </build>
     </profile>
     <profile>
-      <id>java8</id>
-      <activation>
-        <jdk>1.8</jdk>
-      </activation>
-      <properties>
-        <jdk.version>1.8</jdk.version>
-      </properties>
-    </profile>
-    <profile>
       <id>coverage</id>
       <activation>
         <property>
@@ -803,6 +794,7 @@
     <module>asterix-active</module>
     <module>asterix-client-helper</module>
     <module>asterix-license</module>
+    <module>asterix-geo</module>
   </modules>
 
   <dependencyManagement>
@@ -871,6 +863,10 @@
             <groupId>javax.xml.bind</groupId>
             <artifactId>jaxb-api</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>jdk.tools</groupId>
+            <artifactId>jdk.tools</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
@@ -883,6 +879,12 @@
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-mapreduce-client-core</artifactId>
         <version>${hadoop.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>jdk.tools</groupId>
+            <artifactId>jdk.tools</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
@@ -1189,7 +1191,7 @@
       <dependency>
         <groupId>javax.xml.bind</groupId>
         <artifactId>jaxb-api</artifactId>
-        <version>2.2.12</version>
+        <version>2.3.0</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.mojo.appassembler</groupId>
@@ -1207,6 +1209,36 @@
         <artifactId>commons-codec</artifactId>
         <version>1.9</version>
       </dependency>
+      <dependency>
+        <groupId>it.unimi.dsi</groupId>
+        <artifactId>fastutil</artifactId>
+        <version>8.1.1</version>
+      </dependency>
+      <dependency>
+        <groupId>com.sun.xml.bind</groupId>
+        <artifactId>jaxb-core</artifactId>
+        <version>2.3.0</version>
+      </dependency>
+      <dependency>
+        <groupId>com.sun.xml.bind</groupId>
+        <artifactId>jaxb-impl</artifactId>
+        <version>2.3.0</version>
+      </dependency>
+      <dependency>
+        <groupId>javax.activation</groupId>
+        <artifactId>javax.activation-api</artifactId>
+        <version>1.2.0</version>
+      </dependency>
+      <dependency>
+        <groupId>com.sun.activation</groupId>
+        <artifactId>javax.activation</artifactId>
+        <version>1.2.0</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.maven</groupId>
+        <artifactId>maven-core</artifactId>
+        <version>3.3.9</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index c4bdd06..6708f82 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -160,9 +160,9 @@
       <artifactId>netty-all</artifactId>
       <properties>
         <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
-        <license.ignoreMissingEmbeddedLicense>4.1.6.Final</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>4.1.6.Final</license.ignoreMissingEmbeddedNotice>
-        <license.ignoreNoticeOverride>4.1.6.Final</license.ignoreNoticeOverride>
+        <license.ignoreMissingEmbeddedLicense>4.1.25.Final</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>4.1.25.Final</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreNoticeOverride>4.1.25.Final</license.ignoreNoticeOverride>
       </properties>
     </project>
   </supplement>
@@ -208,4 +208,26 @@
       </properties>
     </project>
   </supplement>
+  <supplement>
+    <project>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+      <properties>
+        <!-- esri is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+        <license.ignoreMissingEmbeddedLicense>2.0.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.0.0</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+      <properties>
+        <!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+        <license.ignoreMissingEmbeddedLicense>8.1.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>8.1.1</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
 </supplementalDataModels>
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt
deleted file mode 100644
index 92dfd5b..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt
+++ /dev/null
@@ -1,214 +0,0 @@
-
-                            The Netty Project
-                            =================
-
-Please visit the Netty web site for more information:
-
-  * http://netty.io/
-
-Copyright 2014 The Netty Project
-
-The Netty Project licenses this file to you under the Apache License,
-version 2.0 (the "License"); you may not use this file except in compliance
-with the License. You may obtain a copy of the License at:
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-License for the specific language governing permissions and limitations
-under the License.
-
-Also, please refer to each LICENSE.<component>.txt file, which is located in
-the 'license' directory of the distribution file, for the license terms of the
-components that this product depends on.
-
--------------------------------------------------------------------------------
-This product contains the extensions to Java Collections Framework which has
-been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
-  * LICENSE:
-    * license/LICENSE.jsr166y.txt (Public Domain)
-  * HOMEPAGE:
-    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
-    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-This product contains a modified version of Robert Harder's Public Domain
-Base64 Encoder and Decoder, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.base64.txt (Public Domain)
-  * HOMEPAGE:
-    * http://iharder.sourceforge.net/current/java/base64/
-
-This product contains a modified portion of 'Webbit', an event based  
-WebSocket and HTTP server, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.webbit.txt (BSD License)
-  * HOMEPAGE:
-    * https://github.com/joewalnes/webbit
-
-This product contains a modified portion of 'SLF4J', a simple logging
-facade for Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.slf4j.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.slf4j.org/
-
-This product contains a modified portion of 'Apache Harmony', an open source
-Java SE, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.harmony.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://archive.apache.org/dist/harmony/
-
-This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-and decompression library written by Matthew J. Francis. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jbzip2.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jbzip2/
-
-This product contains a modified portion of 'libdivsufsort', a C API library to construct
-the suffix array and the Burrows-Wheeler transformed string for any input string of
-a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.libdivsufsort.txt (MIT License)
-  * HOMEPAGE:
-    * https://github.com/y-256/libdivsufsort
-
-This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
- which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jctools.txt (ASL2 License)
-  * HOMEPAGE:
-    * https://github.com/JCTools/JCTools
-
-This product optionally depends on 'JZlib', a re-implementation of zlib in
-pure Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jzlib.txt (BSD style License)
-  * HOMEPAGE:
-    * http://www.jcraft.com/jzlib/
-
-This product optionally depends on 'Compress-LZF', a Java library for encoding and
-decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/ning/compress
-
-This product optionally depends on 'lz4', a LZ4 Java compression
-and decompression library written by Adrien Grand. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lz4.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jpountz/lz4-java
-
-This product optionally depends on 'lzma-java', a LZMA Java compression
-and decompression library, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lzma-java.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jponge/lzma-java
-
-This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-and decompression library written by William Kinney. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jfastlz.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jfastlz/
-
-This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-interchange format, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.protobuf.txt (New BSD License)
-  * HOMEPAGE:
-    * https://github.com/google/protobuf
-
-This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
-a temporary self-signed X.509 certificate when the JVM does not provide the
-equivalent functionality.  It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.bouncycastle.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.bouncycastle.org/
-
-This product optionally depends on 'Snappy', a compression library produced
-by Google Inc, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.snappy.txt (New BSD License)
-  * HOMEPAGE:
-    * https://github.com/google/snappy
-
-This product optionally depends on 'JBoss Marshalling', an alternative Java
-serialization API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
-  * HOMEPAGE:
-    * http://www.jboss.org/jbossmarshalling
-
-This product optionally depends on 'Caliper', Google's micro-
-benchmarking framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.caliper.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/google/caliper
-
-This product optionally depends on 'Apache Commons Logging', a logging
-framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-logging.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://commons.apache.org/logging/
-
-This product optionally depends on 'Apache Log4J', a logging framework, which
-can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.log4j.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://logging.apache.org/log4j/
-
-This product optionally depends on 'Aalto XML', an ultra-high performance
-non-blocking XML processor, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://wiki.fasterxml.com/AaltoHome
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.hpack.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/twitter/hpack
-
-This product contains a modified portion of 'Apache Commons Lang', a Java library
-provides utilities for the java.lang API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-lang.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://commons.apache.org/proper/commons-lang/
-
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.25.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.25.Final_NOTICE.txt
new file mode 100644
index 0000000..f973663
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.25.Final_NOTICE.txt
@@ -0,0 +1,223 @@
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project licenses this file to you under the Apache License,
+version 2.0 (the "License"); you may not use this file except in compliance
+with the License. You may obtain a copy of the License at:
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+License for the specific language governing permissions and limitations
+under the License.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * NOTICE:
+    * license/NOTICE.harmony.txt
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://github.com/y-256/libdivsufsort
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/google/protobuf
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/google/snappy
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/google/caliper
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+
+This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
+
+  * LICENSE:
+    * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/takari/maven-wrapper
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index adb6c79..46e80be 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -21,9 +21,11 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
-public class AlgebricksException extends Exception {
+public class AlgebricksException extends Exception implements IFormattedException {
     private static final long serialVersionUID = 1L;
 
     public static final int UNKNOWN = 0;
@@ -31,15 +33,17 @@
     private final int errorCode;
     private final Serializable[] params;
     private final String nodeId;
+    private final SourceLocation sourceLoc;
 
     @SuppressWarnings("squid:S1165") // exception class not final
     private transient CachedMessage msgCache;
 
-    public AlgebricksException(String component, int errorCode, String message, Throwable cause, String nodeId,
-            Serializable... params) {
+    public AlgebricksException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, String nodeId, Serializable... params) {
         super(message, cause);
         this.component = component;
         this.errorCode = errorCode;
+        this.sourceLoc = sourceLoc;
         this.nodeId = nodeId;
         this.params = params;
     }
@@ -49,7 +53,7 @@
      */
     @Deprecated
     public AlgebricksException(String message) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, (Serializable[]) null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
     }
 
     /**
@@ -65,38 +69,68 @@
      */
     @Deprecated
     public AlgebricksException(String message, Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, null, (Serializable[]) null);
+    }
+
+    public AlgebricksException(String component, int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        this(component, errorCode, null, null, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, Serializable... params) {
-        this(component, errorCode, null, null, null, params);
+        this(component, errorCode, null, null, null, null, params);
+    }
+
+    public AlgebricksException(Throwable cause, int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(Throwable cause, int errorCode, Serializable... params) {
-        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
+        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, String message, SourceLocation sourceLoc,
+            Serializable... params) {
+        this(component, errorCode, message, null, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, String message, Serializable... params) {
-        this(component, errorCode, message, null, null, params);
+        this(component, errorCode, message, null, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, Throwable cause, SourceLocation sourceLoc,
+            Serializable... params) {
+        this(component, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, Throwable cause, Serializable... params) {
-        this(component, errorCode, cause.getMessage(), cause, null, params);
+        this(component, errorCode, cause.getMessage(), cause, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, Serializable... params) {
+        this(component, errorCode, message, cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, String message, Throwable cause,
             Serializable... params) {
-        this(component, errorCode, message, cause, null, params);
+        this(component, errorCode, message, cause, null, null, params);
+    }
+
+    public static AlgebricksException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc,
+                params);
     }
 
     public static AlgebricksException create(int errorCode, Serializable... params) {
-        return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+        return create(errorCode, null, params);
     }
 
+    @Override
     public String getComponent() {
         return component;
     }
 
+    @Override
     public int getErrorCode() {
         return errorCode;
     }
@@ -109,11 +143,15 @@
         return nodeId;
     }
 
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
     @Override
     public String getMessage() {
         if (msgCache == null) {
-            msgCache =
-                    new CachedMessage(ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params));
+            msgCache = new CachedMessage(
+                    ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params));
         }
         return msgCache.message;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
index 16bc73e..d308969 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
@@ -22,18 +22,10 @@
     private static final long serialVersionUID = 2L;
 
     public NotImplementedException() {
-        System.err.println("Not implemented.");
+        super("Not implemented.");
     }
 
     public NotImplementedException(String message) {
         super(message);
     }
-
-    public NotImplementedException(Throwable cause) {
-        super(cause);
-    }
-
-    public NotImplementedException(String message, Throwable cause) {
-        super(message, cause);
-    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 339cb15..18aa104 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -36,10 +36,12 @@
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
 @PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
 public class SequentialFirstRuleCheckFixpointRuleControllerTest {
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index 669988c..2a97b6f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -78,6 +78,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
index 7e10203..1de7e3c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILogicalExpression {
 
@@ -73,4 +74,6 @@
     public abstract ILogicalExpression cloneExpression();
 
     public boolean isFunctional();
+
+    SourceLocation getSourceLocation();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
index dd7e065..6bd0d02 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILogicalOperator {
 
@@ -103,4 +104,6 @@
      * Indicates whether the expressions used by this operator must be variable reference expressions.
      */
     public boolean requiresVariableReferenceExpressions();
+
+    SourceLocation getSourceLocation();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
deleted file mode 100644
index 3ecdb35..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.algebricks.core.algebra.base;
-
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public interface ILogicalPlanAndMetadata {
-    public ILogicalPlan getPlan();
-
-    public IMetadataProvider<?, ?> getMetadataProvider();
-
-    public AlgebricksPartitionConstraint getClusterLocations();
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
index 0717c0e..de4d5ac 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
@@ -25,9 +25,12 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractLogicalExpression implements ILogicalExpression {
 
+    protected SourceLocation sourceLoc;
+
     @Override
     public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
             Map<LogicalVariable, EquivalenceClass> equivClasses) {
@@ -45,4 +48,12 @@
         return true;
     }
 
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
index 8fca47c..bdd820e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
@@ -59,6 +59,7 @@
         AggregateFunctionCallExpression fun = new AggregateFunctionCallExpression(finfo, twoStep, clonedArgs);
         fun.setStepTwoAggregate(stepTwoAggregate);
         fun.setStepOneAggregate(stepOneAggregate);
+        fun.setSourceLocation(sourceLoc);
         return fun;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index 689d51c..42ff3c0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -174,6 +174,7 @@
         annotationMap.forEach((key, value1) -> m.put(key, value1.copy()));
         ConstantExpression c = new ConstantExpression(value);
         c.annotationMap = m;
+        c.setSourceLocation(sourceLoc);
         return c;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
index 6308636..f8b25e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
@@ -47,6 +47,7 @@
         ScalarFunctionCallExpression funcExpr = new ScalarFunctionCallExpression(finfo, clonedArgs);
         funcExpr.getAnnotations().putAll(cloneAnnotations());
         funcExpr.setOpaqueParameters(this.getOpaqueParameters());
+        funcExpr.setSourceLocation(sourceLoc);
         return funcExpr;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
index 661c389..0d310f7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
@@ -52,7 +52,10 @@
     public StatefulFunctionCallExpression cloneExpression() {
         cloneAnnotations();
         List<Mutable<ILogicalExpression>> clonedArgs = cloneArguments();
-        return new StatefulFunctionCallExpression(finfo, propertiesComputer, clonedArgs);
+        StatefulFunctionCallExpression clonedExpr =
+                new StatefulFunctionCallExpression(finfo, propertiesComputer, clonedArgs);
+        clonedExpr.setSourceLocation(sourceLoc);
+        return clonedExpr;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
index ac539ec..3dad669 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
@@ -50,6 +50,7 @@
         UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(finfo, clonedArgs);
         ufce.setReturnsUniqueValues(returnsUniqueValues);
         ufce.setOpaqueParameters(this.getOpaqueParameters());
+        ufce.setSourceLocation(sourceLoc);
         return ufce;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
index c1342cc..1c7da34 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
@@ -95,7 +95,9 @@
 
     @Override
     public AbstractLogicalExpression cloneExpression() {
-        return new VariableReferenceExpression(variable);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(variable);
+        varRef.setSourceLocation(sourceLoc);
+        return varRef;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 6860147..ae66ee2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public interface IMetadataProvider<S, I> {
     public IDataSource<S> findDataSource(S id) throws AlgebricksException;
@@ -47,7 +48,8 @@
      */
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException;
 
@@ -206,5 +208,9 @@
             List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKeys,
             RecordDescriptor inputDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException;
 
+    public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+            IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
+            throws AlgebricksException;
+
     public Map<String, String> getConfig();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
index 4686f32..1dbf15e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
 import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractLogicalOperator implements ILogicalOperator {
 
@@ -69,6 +70,8 @@
     protected final List<Mutable<ILogicalOperator>> inputs;
     protected List<LogicalVariable> schema;
 
+    private SourceLocation sourceLoc;
+
     public AbstractLogicalOperator() {
         inputs = new ArrayList<>();
     }
@@ -195,4 +198,13 @@
     public boolean requiresVariableReferenceExpressions() {
         return true;
     }
+
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index 2511fa9..6f11dc1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -35,7 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
 public class DataSourceScanOperator extends AbstractDataSourceOperator {
-    private List<LogicalVariable> projectVars;
+    private final List<LogicalVariable> projectVars;
 
     private boolean projectPushed = false;
 
@@ -43,9 +43,22 @@
     private List<LogicalVariable> minFilterVars;
     private List<LogicalVariable> maxFilterVars;
 
+    // the select condition in the SELECT operator. Only results satisfying this selectCondition
+    // would be returned by this operator
+    private Mutable<ILogicalExpression> selectCondition;
+    // the maximum of number of results output by this operator
+    private long outputLimit = -1;
+
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
+        this(variables, dataSource, null, -1);
+    }
+
+    public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
+            Mutable<ILogicalExpression> selectCondition, long outputLimit) {
         super(variables, dataSource);
         projectVars = new ArrayList<LogicalVariable>();
+        this.selectCondition = selectCondition;
+        this.outputLimit = outputLimit;
     }
 
     @Override
@@ -133,4 +146,20 @@
     public List<Mutable<ILogicalExpression>> getAdditionalFilteringExpressions() {
         return additionalFilteringExpressions;
     }
+
+    public Mutable<ILogicalExpression> getSelectCondition() {
+        return selectCondition;
+    }
+
+    public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+        this.selectCondition = selectCondition;
+    }
+
+    public long getOutputLimit() {
+        return outputLimit;
+    }
+
+    public void setOutputLimit(long outputLimit) {
+        this.outputLimit = outputLimit;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index e7fb6c0..c4bcc52 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -32,9 +32,23 @@
 
 public class UnnestMapOperator extends AbstractUnnestMapOperator {
 
+    // the select condition in the SELECT operator. Only results satisfying this selectCondition
+    // would be returned by this operator
+    private Mutable<ILogicalExpression> selectCondition;
+    // the maximum of number of results output by this operator
+    private long outputLimit = -1;
+
     public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
+        this(variables, expression, variableTypes, propagateInput, null, -1);
+    }
+
+    public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
+            List<Object> variableTypes, boolean propagateInput, Mutable<ILogicalExpression> selectCondition,
+            long outputLimit) {
         super(variables, expression, variableTypes, propagateInput);
+        this.selectCondition = selectCondition;
+        this.outputLimit = outputLimit;
     }
 
     @Override
@@ -64,4 +78,20 @@
         return env;
     }
 
+    public Mutable<ILogicalExpression> getSelectCondition() {
+        return selectCondition;
+    }
+
+    public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+        this.selectCondition = selectCondition;
+    }
+
+    public long getOutputLimit() {
+        return outputLimit;
+    }
+
+    public void setOutputLimit(long outputLimit) {
+        this.outputLimit = outputLimit;
+    }
+
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
index 24c5162..6be8ee0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IVariableContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
@@ -80,6 +81,10 @@
         dest.setOpaqueParameters(newOpaqueParameters);
     }
 
+    private void copySourceLocation(ILogicalExpression src, AbstractLogicalExpression dest) {
+        dest.setSourceLocation(src.getSourceLocation());
+    }
+
     public MutableObject<ILogicalExpression> deepCopyExpressionReference(Mutable<ILogicalExpression> exprRef)
             throws AlgebricksException {
         return new MutableObject<>(deepCopy(exprRef.getValue()));
@@ -104,12 +109,15 @@
         exprCopy.setStepTwoAggregate(expr.getStepTwoAggregate());
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
     @Override
     public ILogicalExpression visitConstantExpression(ConstantExpression expr, Void arg) throws AlgebricksException {
-        return new ConstantExpression(expr.getValue());
+        ConstantExpression exprCopy = new ConstantExpression(expr.getValue());
+        copySourceLocation(expr, exprCopy);
+        return exprCopy;
     }
 
     @Override
@@ -119,6 +127,7 @@
                 deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
 
     }
@@ -130,6 +139,7 @@
                 expr.getPropertiesComputer(), deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
@@ -140,6 +150,7 @@
                 deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
@@ -153,13 +164,17 @@
         LogicalVariable givenVarReplacement = inVarMapping.get(var);
         if (givenVarReplacement != null) {
             outVarMapping.put(var, givenVarReplacement);
-            return new VariableReferenceExpression(givenVarReplacement);
+            VariableReferenceExpression varRef = new VariableReferenceExpression(givenVarReplacement);
+            copySourceLocation(expr, varRef);
+            return varRef;
         }
         LogicalVariable varCopy = outVarMapping.get(var);
         if (varCopy == null) {
             varCopy = varContext.newVar();
             outVarMapping.put(var, varCopy);
         }
-        return new VariableReferenceExpression(varCopy);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(varCopy);
+        copySourceLocation(expr, varRef);
+        return varRef;
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 74afdf5..e0210cc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -146,6 +146,10 @@
         dest.getAnnotations().putAll(src.getAnnotations());
     }
 
+    private void copySourceLocation(ILogicalOperator src, AbstractLogicalOperator dest) {
+        dest.setSourceLocation(src.getSourceLocation());
+    }
+
     public ILogicalOperator deepCopy(ILogicalOperator op) throws AlgebricksException {
         // The deep copy call outside this visitor always has a null argument.
         return deepCopy(op, null);
@@ -269,6 +273,7 @@
             AbstractLogicalOperator opCopy) throws AlgebricksException {
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
     }
 
@@ -317,8 +322,10 @@
     @Override
     public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg)
             throws AlgebricksException {
-        DataSourceScanOperator opCopy =
-                new DataSourceScanOperator(deepCopyVariableList(op.getVariables()), op.getDataSource());
+        Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+                ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
+        DataSourceScanOperator opCopy = new DataSourceScanOperator(deepCopyVariableList(op.getVariables()),
+                op.getDataSource(), newSelectCondition, op.getOutputLimit());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
@@ -335,6 +342,7 @@
     @Override
     public ILogicalOperator visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg) {
         EmptyTupleSourceOperator opCopy = new EmptyTupleSourceOperator();
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -369,6 +377,7 @@
                         deepCopyOperatorReference(op.getInputs().get(0), arg),
                         deepCopyOperatorReference(op.getInputs().get(1), arg));
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -381,6 +390,7 @@
                         deepCopyOperatorReference(op.getInputs().get(0), arg),
                         deepCopyOperatorReference(op.getInputs().get(1), arg));
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -535,9 +545,11 @@
     @Override
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg)
             throws AlgebricksException {
+        Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+                ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
         UnnestMapOperator opCopy = new UnnestMapOperator(deepCopyVariableList(op.getVariables()),
                 exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
-                op.propagatesInput());
+                op.propagatesInput(), newSelectCondition, op.getOutputLimit());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 600714b..0db0f74 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -35,11 +35,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -225,8 +225,10 @@
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
         ArrayList<LogicalVariable> newInputList = new ArrayList<>();
         newInputList.addAll(op.getVariables());
+        Mutable<ILogicalExpression> newSelectCondition =
+                op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
         return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
-                new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+                new ArrayList<>(op.getVariableTypes()), op.propagatesInput(), newSelectCondition, op.getOutputLimit());
     }
 
     @Override
@@ -242,7 +244,11 @@
     public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
         ArrayList<LogicalVariable> newInputList = new ArrayList<>();
         newInputList.addAll(op.getVariables());
-        return new DataSourceScanOperator(newInputList, op.getDataSource());
+        Mutable<ILogicalExpression> newSelectCondition =
+                op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
+        DataSourceScanOperator newOp =
+                new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit());
+        return newOp;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index cf24ee7..3587e29 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -34,11 +34,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -134,6 +134,9 @@
             }
         }
         substVarTypes(op, pair);
+        if (op.getSelectCondition() != null) {
+            op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
+        }
         return null;
     }
 
@@ -316,6 +319,9 @@
     public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         substituteVarsForAbstractUnnestMapOp(op, pair);
+        if (op.getSelectCondition() != null) {
+            op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index b8cb4ff..d57a998 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -19,7 +19,9 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
 
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -34,11 +36,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -76,7 +78,7 @@
 
 public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
 
-    private Collection<LogicalVariable> usedVariables;
+    private final Collection<LogicalVariable> usedVariables;
 
     public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
         this.usedVariables = usedVariables;
@@ -105,6 +107,12 @@
                 e.getValue().getUsedVariables(usedVariables);
             }
         }
+        if (op.getSelectCondition() != null) {
+            Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+            op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+            usedVariablesBySelect.removeAll(op.getVariables());
+            usedVariables.addAll(usedVariablesBySelect);
+        }
         return null;
     }
 
@@ -305,6 +313,12 @@
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {
         getUsedVarsFromExprAndFilterExpr(op);
+        if (op.getSelectCondition() != null) {
+            Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+            op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+            usedVariablesBySelect.removeAll(op.getVariables());
+            usedVariables.addAll(usedVariablesBySelect);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 4a177f7..0602258 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -81,6 +81,7 @@
         int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
         UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
                 unnest.getPositionWriter(), leftOuter, context.getMissingWriterFactory());
+        unnestRuntime.setSourceLocation(unnest.getSourceLocation());
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
         ILogicalOperator src = unnest.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
index 147d5cc..2736b6f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
@@ -102,6 +102,7 @@
         }
 
         AggregateRuntimeFactory runtime = new AggregateRuntimeFactory(aggFactories);
+        runtime.setSourceLocation(aggOp.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
index ccd27f4..2df5a4e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
@@ -90,6 +90,7 @@
 
         AssignRuntimeFactory runtime =
                 new AssignRuntimeFactory(outColumns, evalFactories, projectionList, flushFramesRapidly);
+        runtime.setSourceLocation(assign.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
@@ -103,7 +104,6 @@
         // and contribute one edge from its child
         ILogicalOperator src = assign.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, assign, 0);
-
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
index 2204637..bb0f08b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
@@ -107,8 +107,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, primaryKeys, payload,
                         additionalFilteringKeys, additionalNonFilterVars, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 1421cef..3ddb233 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -43,11 +43,12 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 @SuppressWarnings("rawtypes")
 public class DataSourceScanPOperator extends AbstractScanPOperator {
 
-    private IDataSource<?> dataSource;
+    private final IDataSource<?> dataSource;
     private Object implConfig;
 
     public DataSourceScanPOperator(IDataSource<?> dataSource) {
@@ -109,12 +110,20 @@
         List<LogicalVariable> vars = scan.getVariables();
         List<LogicalVariable> projectVars = scan.getProjectVariables();
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p =
-                mp.getScannerRuntime(dataSource, vars, projectVars, scan.isProjectPushed(), scan.getMinFilterVars(),
-                        scan.getMaxFilterVars(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
-        builder.contributeHyracksOperator(scan, p.first);
+        ITupleFilterFactory tupleFilterFactory = null;
+        if (scan.getSelectCondition() != null) {
+            tupleFilterFactory = context.getMetadataProvider().createTupleFilterFactory(
+                    new IOperatorSchema[] { opSchema }, typeEnv, scan.getSelectCondition().getValue(), context);
+        }
+
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars, projectVars,
+                scan.isProjectPushed(), scan.getMinFilterVars(), scan.getMaxFilterVars(), tupleFilterFactory,
+                scan.getOutputLimit(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
+        IOperatorDescriptor opDesc = p.first;
+        opDesc.setSourceLocation(scan.getSourceLocation());
+        builder.contributeHyracksOperator(scan, opDesc);
         if (p.second != null) {
-            builder.contributeAlgebricksPartitionConstraint(p.first, p.second);
+            builder.contributeAlgebricksPartitionConstraint(opDesc, p.second);
         }
 
         ILogicalOperator srcExchange = scan.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
index 178f2a1..34dc0d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -105,8 +105,10 @@
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getResultHandleRuntime(resultOp.getDataSink(), columns, pf, inputDesc, true, spec);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(resultOp.getSourceLocation());
+        builder.contributeHyracksOperator(resultOp, opDesc);
 
-        builder.contributeHyracksOperator(resultOp, runtimeAndConstraints.first);
         ILogicalOperator src = resultOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, resultOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
index 0718d13..502d022 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
@@ -63,6 +63,7 @@
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         EmptyTupleSourceRuntimeFactory runtime = new EmptyTupleSourceRuntimeFactory();
+        runtime.setSourceLocation(op.getSourceLocation());
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
         builder.contributeMicroOperator(op, runtime, recDesc);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
index 5ee967d..edc9d08 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -62,6 +62,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
@@ -253,8 +254,10 @@
             merges[i] = expressionRuntimeProvider.createSerializableAggregateFunctionFactory(mergeFun, aggOpInputEnv,
                     localInputSchemas, context);
         }
-        IAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
-        IAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
+        AbstractAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
+        AbstractAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
+        mergeFactory.setSourceLocation(gby.getSourceLocation());
 
         INormalizedKeyComputerFactory normalizedKeyFactory =
                 JobGenHelper.variablesToAscNormalizedKeyComputerFactory(gbyCols, aggOpInputEnv, context);
@@ -268,6 +271,7 @@
         ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, hashTableSize, inputSize,
                 keyAndDecFields, frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
                 recordDescriptor, recordDescriptor, new HashSpillableTableFactory(hashFunctionFactories));
+        gbyOpDesc.setSourceLocation(gby.getSourceLocation());
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index 301b8f1..d06dde5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -152,6 +152,7 @@
             opDesc = generateHashJoinRuntime(context, inputSchemas, keysLeft, keysRight, hashFunFactories,
                     comparatorFactories, predEvaluatorFactory, recDescriptor, spec);
         }
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
@@ -164,68 +165,49 @@
             int[] keysLeft, int[] keysRight, IBinaryHashFunctionFactory[] hashFunFactories,
             IBinaryComparatorFactory[] comparatorFactories, IPredicateEvaluatorFactory predEvaluatorFactory,
             RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) throws AlgebricksException {
-        IOperatorDescriptor opDesc;
-        try {
-            switch (kind) {
-                case INNER:
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
-                            aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
-                            comparatorFactories, recDescriptor, predEvaluatorFactory, false, null);
-                    break;
-                case LEFT_OUTER:
-                    IMissingWriterFactory[] nonMatchWriterFactories =
-                            new IMissingWriterFactory[inputSchemas[1].getSize()];
-                    for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                        nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                    }
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
-                            aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
-                            comparatorFactories, recDescriptor, predEvaluatorFactory, true, nonMatchWriterFactories);
-                    break;
-                default:
-                    throw new NotImplementedException();
-            }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        switch (kind) {
+            case INNER:
+                return new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
+                        aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
+                        comparatorFactories, recDescriptor, predEvaluatorFactory, false, null);
+            case LEFT_OUTER:
+                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
+                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
+                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
+                }
+                return new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
+                        aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
+                        comparatorFactories, recDescriptor, predEvaluatorFactory, true, nonMatchWriterFactories);
+            default:
+                throw new NotImplementedException();
         }
-        return opDesc;
     }
 
     private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context, IOperatorSchema[] inputSchemas,
             int[] keysLeft, int[] keysRight, IBinaryHashFunctionFamily[] hashFunFamilies,
             IBinaryComparatorFactory[] comparatorFactories, IPredicateEvaluatorFactory predEvaluatorFactory,
             RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) throws AlgebricksException {
-        IOperatorDescriptor opDesc;
-        try {
-            switch (kind) {
-                case INNER:
-                    opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
-                            comparatorFactories, recDescriptor,
-                            new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
-                            new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft),
-                            predEvaluatorFactory);
-                    break;
-                case LEFT_OUTER:
-                    IMissingWriterFactory[] nonMatchWriterFactories =
-                            new IMissingWriterFactory[inputSchemas[1].getSize()];
-                    for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                        nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                    }
-                    opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
-                            comparatorFactories, recDescriptor,
-                            new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
-                            new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft),
-                            predEvaluatorFactory, true, nonMatchWriterFactories);
-                    break;
-                default:
-                    throw new NotImplementedException();
-            }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        switch (kind) {
+            case INNER:
+                return new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                        maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                        comparatorFactories, recDescriptor,
+                        new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
+                        new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft), predEvaluatorFactory);
+            case LEFT_OUTER:
+                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
+                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
+                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
+                }
+                return new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                        maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                        comparatorFactories, recDescriptor,
+                        new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
+                        new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft), predEvaluatorFactory,
+                        true, nonMatchWriterFactories);
+            default:
+                throw new NotImplementedException();
         }
-        return opDesc;
     }
 
     @Override
@@ -264,7 +246,7 @@
 }
 
 /**
- * {@ ITuplePairComparatorFactory} implementation for optimized hybrid hash join.
+ * {@code ITuplePairComparatorFactory} implementation for optimized hybrid hash join.
  */
 class JoinMultiComparatorFactory implements ITuplePairComparatorFactory {
     private static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 9c29c53..580d8e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -105,15 +105,14 @@
         RecordDescriptor recDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
-        IOperatorDescriptor opDesc = null;
+        IOperatorDescriptor opDesc;
 
         switch (kind) {
-            case INNER: {
+            case INNER:
                 opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
                         comparatorFactories, recDescriptor, tableSize, predEvaluatorFactory, memSizeInFrames);
                 break;
-            }
-            case LEFT_OUTER: {
+            case LEFT_OUTER:
                 IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
                 for (int j = 0; j < nonMatchWriterFactories.length; j++) {
                     nonMatchWriterFactories[j] = context.getMissingWriterFactory();
@@ -122,11 +121,11 @@
                         comparatorFactories, predEvaluatorFactory, recDescriptor, true, nonMatchWriterFactories,
                         tableSize, memSizeInFrames);
                 break;
-            }
-            default: {
+            default:
                 throw new NotImplementedException();
-            }
         }
+
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index fa0fb1a..6512700 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -135,8 +135,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
                         secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(indexInsertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(indexInsertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(indexInsertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = indexInsertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, indexInsertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index a66db35..92fa86d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -118,19 +118,30 @@
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(insertDeleteUpsertOp);
-        if (insertDeleteUpsertOp.getOperation() == Kind.INSERT) {
-            runtimeAndConstraints = mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
-        } else if (insertDeleteUpsertOp.getOperation() == Kind.DELETE) {
-            runtimeAndConstraints = mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
-        } else if (insertDeleteUpsertOp.getOperation() == Kind.UPSERT) {
-            runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, prevSecondaryKeys,
-                    prevAdditionalFilteringKey, inputDesc, context, spec);
+        Kind operation = insertDeleteUpsertOp.getOperation();
+        switch (operation) {
+            case INSERT:
+                runtimeAndConstraints =
+                        mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
+                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
+                break;
+            case DELETE:
+                runtimeAndConstraints =
+                        mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
+                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
+                break;
+            case UPSERT:
+                runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+                        typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, prevSecondaryKeys,
+                        prevAdditionalFilteringKey, inputDesc, context, spec);
+                break;
+            default:
+                throw new AlgebricksException("Unsupported Operation " + operation);
         }
-        builder.contributeHyracksOperator(insertDeleteUpsertOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteUpsertOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteUpsertOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteUpsertOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteUpsertOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
index 6ded4a3..927fb66 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
@@ -106,22 +106,28 @@
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
-        if (operation == Kind.INSERT) {
-            runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
-                    additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec, false);
-        } else if (operation == Kind.DELETE) {
-            runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
-                    additionalFilteringKeys, inputDesc, context, spec);
-        } else if (operation == Kind.UPSERT) {
-            runtimeAndConstraints = mp.getUpsertRuntime(dataSource, inputSchemas[0], typeEnv, keys, payload,
-                    additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec);
-        } else {
-            throw new AlgebricksException("Unsupported Operation " + operation);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints;
+        switch (operation) {
+            case INSERT:
+                runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec, false);
+                break;
+            case DELETE:
+                runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, inputDesc, context, spec);
+                break;
+            case UPSERT:
+                runtimeAndConstraints = mp.getUpsertRuntime(dataSource, inputSchemas[0], typeEnv, keys, payload,
+                        additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec);
+                break;
+            default:
+                throw new AlgebricksException("Unsupported Operation " + operation);
         }
 
-        builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
index 1d36cc0..544c546 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
@@ -151,7 +151,7 @@
         } catch (HyracksException e) {
             throw new AlgebricksException(e);
         }
-
+        opDescriptor.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDescriptor);
         for (int i = 0; i < op.getInputs().size(); i++) {
             builder.contributeGraphEdge(op.getInputs().get(i).getValue(), 0, op, i);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
index a48e3c2..fe8985a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
@@ -74,6 +74,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         MaterializingOperatorDescriptor materializationOpDesc =
                 new MaterializingOperatorDescriptor(builder.getJobSpec(), recDescriptor, isSingleActivity);
+        materializationOpDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
index 94d5fd4..fda879c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 
 public class MicroPreSortedDistinctByPOperator extends AbstractPreSortedDistinctByPOperator {
 
@@ -62,8 +62,9 @@
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
                 .variablesToAscBinaryComparatorFactories(columnList, context.getTypeEnvironment(op), context);
         IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, keysAndDecs);
+        aggregatorFactory.setSourceLocation(op.getSourceLocation());
 
         RecordDescriptor recordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
@@ -73,6 +74,7 @@
         /* make fd columns part of the key but the comparator only compares the distinct key columns */
         MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keysAndDecs,
                 comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recordDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
index 629afa3..13308a1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
@@ -74,6 +74,7 @@
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
         MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keys,
                 comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
+        runtime.setSourceLocation(gby.getSourceLocation());
         builder.contributeMicroOperator(gby, runtime, recordDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
index f5e992e..da48049 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
@@ -49,6 +49,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
 
         MicroUnionAllRuntimeFactory runtime = new MicroUnionAllRuntimeFactory(op.getInputs().size());
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recordDescriptor);
 
         super.contributeRuntimeOperator(builder, context, op, opSchema, inputSchemas, outerPlanSchema);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 4d7bd7e1..83cea4a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -139,15 +139,14 @@
         ITuplePairComparatorFactory comparatorFactory =
                 new TuplePairEvaluatorFactory(cond, context.getBinaryBooleanInspectorFactory());
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
-        IOperatorDescriptor opDesc = null;
+        IOperatorDescriptor opDesc;
 
         switch (kind) {
-            case INNER: {
+            case INNER:
                 opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, false,
                         null);
                 break;
-            }
-            case LEFT_OUTER: {
+            case LEFT_OUTER:
                 IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
                 for (int j = 0; j < nonMatchWriterFactories.length; j++) {
                     nonMatchWriterFactories[j] = context.getMissingWriterFactory();
@@ -155,12 +154,12 @@
                 opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
                         nonMatchWriterFactories);
                 break;
-            }
-            default: {
+            default:
                 throw new NotImplementedException();
-            }
         }
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+        opDesc.setSourceLocation(join.getSourceLocation());
+        contributeOpDesc(builder, join, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src1, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
index 179bb73..7971e78 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
@@ -98,6 +98,7 @@
             throws AlgebricksException {
         propagatedSchema.addAllVariables(outerPlanSchema);
         NestedTupleSourceRuntimeFactory runtime = new NestedTupleSourceRuntimeFactory();
+        runtime.setSourceLocation(op.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(op, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
index 54e577f..dd4c65f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 
 public class PreSortedDistinctByPOperator extends AbstractPreSortedDistinctByPOperator {
@@ -64,14 +64,16 @@
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
                 .variablesToAscBinaryComparatorFactories(columnList, context.getTypeEnvironment(op), context);
         IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, keysAndDecs);
+        aggregatorFactory.setSourceLocation(op.getSourceLocation());
 
         RecordDescriptor recordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         /* make fd columns part of the key but the comparator only compares the distinct key columns */
         PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keysAndDecs,
                 comparatorFactories, aggregatorFactory, recordDescriptor);
+        opDesc.setSourceLocation(op.getSourceLocation());
 
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
index 0e0953c..e5076ce 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -38,7 +37,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 
 public class PreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
@@ -71,7 +70,7 @@
         int fdColumns[] = getFdColumns(gby, inputSchemas[0]);
         // compile subplans and set the gby op. schema accordingly
         AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
-        IAggregatorDescriptorFactory aggregatorFactory;
+        AbstractAggregatorDescriptorFactory aggregatorFactory;
 
         List<ILogicalPlan> nestedPlans = gby.getNestedPlans();
         if (!nestedPlans.isEmpty() && nestedPlans.get(0).getRoots().get(0).getValue()
@@ -80,6 +79,7 @@
         } else {
             aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys, fdColumns);
         }
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
 
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
@@ -89,8 +89,9 @@
 
         PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
                 comparatorFactories, aggregatorFactory, recordDescriptor, groupAll, framesLimit);
+        opDesc.setSourceLocation(gby.getSourceLocation());
 
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+        contributeOpDesc(builder, gby, opDesc);
 
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
index 25d31d2..f34cecc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
@@ -22,7 +22,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -50,9 +49,10 @@
         int outputArity = rop.getOutputArity();
         boolean[] outputMaterializationFlags = rop.getOutputMaterializationFlags();
 
-        ReplicateOperatorDescriptor splitOpDesc =
+        ReplicateOperatorDescriptor ropDesc =
                 new ReplicateOperatorDescriptor(spec, recDescriptor, outputArity, outputMaterializationFlags);
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, splitOpDesc);
+        ropDesc.setSourceLocation(rop.getSourceLocation());
+        contributeOpDesc(builder, rop, ropDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index 3a6ba74..d68be20 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -91,6 +91,7 @@
 
         RunningAggregateRuntimeFactory runtime =
                 new RunningAggregateRuntimeFactory(outColumns, runningAggFuns, projectionList);
+        runtime.setSourceLocation(ragg.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
index 5084c18..6c9f6e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
@@ -79,6 +79,7 @@
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
 
         SinkOperatorDescriptor opDesc = new SinkOperatorDescriptor(spec, op.getInputs().size());
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         for (int i = 0; i < op.getInputs().size(); i++) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index f76b69b..6e37b3a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -101,10 +101,12 @@
 
         IMetadataProvider<?, ?> mp = context.getMetadataProvider();
 
-        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtime =
+        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+        IPushRuntimeFactory runtime = runtimeAndConstraints.first;
+        runtime.setSourceLocation(write.getSourceLocation());
 
-        builder.contributeMicroOperator(write, runtime.first, recDesc, runtime.second);
+        builder.contributeMicroOperator(write, runtime, recDesc, runtimeAndConstraints.second);
         ILogicalOperator src = write.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, write, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
index 1aeeca9..967e7e6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
@@ -63,7 +63,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
 
 public class SortGroupByPOperator extends AbstractPhysicalOperator {
@@ -258,10 +258,12 @@
         RecordDescriptor partialAggRecordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), localInputSchemas[0], context);
 
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aff, keyAndDecFields);
-        IAggregatorDescriptorFactory mergeFactory =
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
+        AbstractAggregatorDescriptorFactory mergeFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(merges, keyAndDecFields);
+        mergeFactory.setSourceLocation(gby.getSourceLocation());
 
         INormalizedKeyComputerFactory normalizedKeyFactory = null;
         INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
@@ -275,6 +277,7 @@
         SortGroupByOperatorDescriptor gbyOpDesc = new SortGroupByOperatorDescriptor(spec, frameLimit, keys,
                 keyAndDecFields, normalizedKeyFactory, compFactories, aggregatorFactory, mergeFactory,
                 partialAggRecordDescriptor, recordDescriptor, false);
+        gbyOpDesc.setSourceLocation(gby.getSourceLocation());
 
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
index c9fde4b..2f02eba 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
@@ -60,10 +60,11 @@
 
         IBinaryIntegerInspectorFactory intInsepctorFactory = context.getBinaryIntegerInspectorFactory();
 
-        SplitOperatorDescriptor splitOpDesc = new SplitOperatorDescriptor(spec, recDescriptor, outputArity,
+        SplitOperatorDescriptor sopDesc = new SplitOperatorDescriptor(spec, recDescriptor, outputArity,
                 brachingExprEvalFactory, intInsepctorFactory, defaultBranch, propageToAllBranchAsDefault);
+        sopDesc.setSourceLocation(sop.getSourceLocation());
 
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, splitOpDesc);
+        contributeOpDesc(builder, sop, sopDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
index 3a4249b..269a809 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
 
@@ -98,22 +99,21 @@
             i++;
         }
 
+        AbstractSorterOperatorDescriptor sortOpDesc;
         // topK == -1 means that a topK value is not provided.
         if (topK == -1) {
-            ExternalSortOperatorDescriptor sortOpDesc =
+            sortOpDesc =
                     new ExternalSortOperatorDescriptor(spec, maxNumberOfFrames, sortFields, nkcf, comps, recDescriptor);
-            contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
-            ILogicalOperator src = op.getInputs().get(0).getValue();
-            builder.contributeGraphEdge(src, 0, op, 0);
         } else {
             // Since topK value is provided, topK optimization is possible.
             // We call topKSorter instead of calling ExternalSortOperator.
-            TopKSorterOperatorDescriptor sortOpDesc = new TopKSorterOperatorDescriptor(spec, maxNumberOfFrames, topK,
-                    sortFields, nkcf, comps, recDescriptor);
-            contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
-            ILogicalOperator src = op.getInputs().get(0).getValue();
-            builder.contributeGraphEdge(src, 0, op, 0);
+            sortOpDesc = new TopKSorterOperatorDescriptor(spec, maxNumberOfFrames, topK, sortFields, nkcf, comps,
+                    recDescriptor);
         }
+        sortOpDesc.setSourceLocation(op.getSourceLocation());
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
index da75da8..90732ce 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
@@ -98,6 +98,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         StreamLimitRuntimeFactory runtime = new StreamLimitRuntimeFactory(maxObjectsFact, offsetFact, null,
                 context.getBinaryIntegerInspectorFactory());
+        runtime.setSourceLocation(limit.getSourceLocation());
         builder.contributeMicroOperator(limit, runtime, recDesc);
         // and contribute one edge from its child
         ILogicalOperator src = limit.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
index 3ff7dc1..0789cee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
@@ -68,6 +68,7 @@
             projectionList[i++] = pos;
         }
         StreamProjectRuntimeFactory runtime = new StreamProjectRuntimeFactory(projectionList, flushFramesRapidly);
+        runtime.setSourceLocation(project.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(project, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index ddde5f3..a519275 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -70,6 +70,7 @@
                 new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(),
                         select.getRetainMissing(), inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()),
                         context.getMissingWriterFactory());
+        runtime.setSourceLocation(select.getSourceLocation());
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(select, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
index 01e9a0c..0e8005d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
@@ -65,6 +65,7 @@
         StringStreamingScriptDescription sssd = (StringStreamingScriptDescription) scriptDesc;
         StringStreamingRuntimeFactory runtime = new StringStreamingRuntimeFactory(sssd.getCommand(),
                 sssd.getPrinterFactories(), sssd.getFieldDelimiter(), sssd.getParserFactory());
+        runtime.setSourceLocation(scriptOp.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(scriptOp, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
index 95efbac..5ec6d0a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -101,6 +101,7 @@
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         SubplanRuntimeFactory runtime =
                 new SubplanRuntimeFactory(np, missingWriterFactories, inputRecordDesc, recDesc, null);
+        runtime.setSourceLocation(subplan.getSourceLocation());
         builder.contributeMicroOperator(subplan, runtime, recDesc);
 
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
index cd696bc..57f8f42 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
@@ -94,8 +94,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getTokenizerRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
                         secondaryKeys, null, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(tokenizeOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(tokenizeOp.getSourceLocation());
+        builder.contributeHyracksOperator(tokenizeOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = tokenizeOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, tokenizeOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
index 4ccce92..fcd9fa7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -50,6 +50,7 @@
 
         UnionAllOperatorDescriptor opDesc =
                 new UnionAllOperatorDescriptor(builder.getJobSpec(), op.getInputs().size(), recordDescriptor);
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         super.contributeRuntimeOperator(builder, context, op, opSchema, inputSchemas, outerPlanSchema);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
index 7ec1914..70e596a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
@@ -101,9 +101,10 @@
         JobSpecification spec = builder.getJobSpec();
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getWriteResultRuntime(
                 dataSource, propagatedSchema, keys, payload, additionalFilteringKeys, context, spec);
-
-        builder.contributeHyracksOperator(writeResultOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(writeResultOp.getSourceLocation());
+        builder.contributeHyracksOperator(writeResultOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = writeResultOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, writeResultOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 70f19c1..99ed738 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -299,21 +299,25 @@
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit());
+        return null;
     }
 
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
             throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        return null;
     }
 
-    private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
-            throws AlgebricksException {
+    private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+            String opSignature) throws AlgebricksException {
         AlgebricksAppendable plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
                 + op.getExpressionRef().getValue().accept(exprVisitor, indent));
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
-        return null;
+        return plan;
     }
 
     @Override
@@ -321,6 +325,24 @@
         AlgebricksAppendable plan = addIndent(indent).append(
                 "data-scan " + op.getProjectVariables() + "<-" + op.getVariables() + " <- " + op.getDataSource());
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit());
+        return null;
+    }
+
+    private Void appendSelectConditionInformation(AlgebricksAppendable plan,
+            Mutable<ILogicalExpression> selectCondition, Integer indent) throws AlgebricksException {
+        if (selectCondition != null) {
+            plan.append(" condition (").append(selectCondition.getValue().accept(exprVisitor, indent)).append(")");
+        }
+
+        return null;
+    }
+
+    private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit) throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(" limit ").append(String.valueOf(outputLimit));
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 8acf08b..f1f1f3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -83,7 +83,7 @@
 
     public class IdCounter {
         private int id;
-        private Deque<Integer> prefix;
+        private final Deque<Integer> prefix;
 
         public IdCounter() {
             prefix = new LinkedList<Integer>();
@@ -400,24 +400,28 @@
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit(), indent);
+        return null;
     }
 
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
             throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        return null;
     }
 
-    private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
-            throws AlgebricksException {
+    private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+            String opSignature) throws AlgebricksException {
         AlgebricksAppendable plan = addIndent(indent).append("\"operator\": \"" + opSignature + "\"");
         variablePrintHelper(op.getVariables(), indent);
         buffer.append(",\n");
         addIndent(indent).append("\"expressions\": \""
                 + op.getExpressionRef().getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
-        return null;
+        return plan;
     }
 
     @Override
@@ -435,6 +439,8 @@
             addIndent(indent).append("\"data-source\": \"" + op.getDataSource() + "\"");
         }
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit(), indent);
         return null;
     }
 
@@ -467,6 +473,25 @@
         return null;
     }
 
+    private Void appendSelectConditionInformation(AlgebricksAppendable plan, Mutable<ILogicalExpression> condition,
+            Integer indent) throws AlgebricksException {
+        if (condition != null) {
+            plan.append(",\n");
+            addIndent(indent).append(
+                    "\"condition\": \"" + condition.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+        }
+        return null;
+    }
+
+    private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit, Integer indent)
+            throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(",\n");
+            addIndent(indent).append("\"limit\": \"" + outputLimit + "\"");
+        }
+        return null;
+    }
+
     private void appendVars(List<LogicalVariable> minFilterVars) throws AlgebricksException {
         boolean first = true;
         for (LogicalVariable v : minFilterVars) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index c574cd8..67199b9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -262,6 +262,7 @@
     public static ILogicalOperator deepCopy(ILogicalOperator op) throws AlgebricksException {
         OperatorDeepCopyVisitor visitor = new OperatorDeepCopyVisitor();
         AbstractLogicalOperator copiedOperator = (AbstractLogicalOperator) op.accept(visitor, null);
+        copiedOperator.setSourceLocation(op.getSourceLocation());
         copiedOperator.setExecutionMode(op.getExecutionMode());
         copiedOperator.getAnnotations().putAll(op.getAnnotations());
         copiedOperator.setSchema(op.getSchema());
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 4649d6d..2cb2d35 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -289,21 +289,24 @@
     @Override
     public String visitUnnestMapOperator(UnnestMapOperator op, Void noArgs) throws AlgebricksException {
         stringBuilder.setLength(0);
-        return printAbstractUnnestMapOperator(op, "unnest-map");
+        printAbstractUnnestMapOperator(op, "unnest-map");
+        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        return stringBuilder.toString();
     }
 
     @Override
     public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void noArgs)
             throws AlgebricksException {
         stringBuilder.setLength(0);
-        return printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        return stringBuilder.toString();
     }
 
-    private String printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
+    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
         stringBuilder.append(opSignature).append(" ").append(op.getVariables()).append(" <- ")
                 .append(op.getExpressionRef().getValue().toString());
         appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
-        return stringBuilder.toString();
     }
 
     @Override
@@ -312,10 +315,12 @@
         stringBuilder.append("data-scan ").append(op.getProjectVariables()).append("<-").append(op.getVariables())
                 .append(" <- ").append(op.getDataSource());
         appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+        appendLimitInformation(stringBuilder, op.getOutputLimit());
         return stringBuilder.toString();
     }
 
-    private String appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
+    private void appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
             List<LogicalVariable> maxFilterVars) {
         if (minFilterVars != null || maxFilterVars != null) {
             plan.append(" with filter on");
@@ -326,7 +331,21 @@
         if (maxFilterVars != null) {
             plan.append(" max:").append(maxFilterVars);
         }
-        return stringBuilder.toString();
+    }
+
+    private Void appendSelectConditionInformation(StringBuilder plan, Mutable<ILogicalExpression> condition)
+            throws AlgebricksException {
+        if (condition != null) {
+            plan.append(" condition:").append(condition.getValue().toString());
+        }
+        return null;
+    }
+
+    private Void appendLimitInformation(StringBuilder plan, long outputLimit) throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(" limit:").append(String.valueOf(outputLimit));
+        }
+        return null;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
index 24a5e21..d507e5a 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.algebricks.data;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public interface IBinaryBooleanInspector {
-    public boolean getBooleanValue(byte[] bytes, int offset, int length);
+    public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
index d9dccce..dd1cbe6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
@@ -49,6 +49,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
index 7802b29..7fc2efe 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractDecorrelationRule implements IAlgebraicRewriteRule {
 
@@ -91,9 +92,11 @@
 
     protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g,
             List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+        SourceLocation sourceLoc = g.getSourceLocation();
         for (LogicalVariable ov : vars) {
             LogicalVariable newVar = context.newVar();
-            ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+            VariableReferenceExpression varExpr = new VariableReferenceExpression(newVar);
+            varExpr.setSourceLocation(sourceLoc);
             outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
                     new MutableObject<ILogicalExpression>(varExpr)));
             for (ILogicalPlan p : g.getNestedPlans()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
index bce72c1..036b3e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
@@ -36,6 +36,7 @@
             IOptimizationContext context) throws AlgebricksException {
         LogicalVariable v = context.newVar();
         AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(gExpr));
+        a.setSourceLocation(gExpr.getSourceLocation());
         a.getInputs().add(new MutableObject<ILogicalOperator>(opRef2.getValue()));
         opRef2.setValue(a);
         if (gExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
index 3f61cc0..ade2402 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractIntroduceCombinerRule implements IAlgebraicRewriteRule {
 
@@ -67,7 +68,7 @@
 
     protected Pair<Boolean, Mutable<ILogicalOperator>> tryToPushAgg(AggregateOperator initAgg, GroupByOperator newGbyOp,
             Set<SimilarAggregatesInfo> toReplaceSet, IOptimizationContext context) throws AlgebricksException {
-
+        SourceLocation sourceLoc = initAgg.getSourceLocation();
         List<LogicalVariable> initVars = initAgg.getVariables();
         List<Mutable<ILogicalExpression>> initExprs = initAgg.getExpressions();
         int numExprs = initVars.size();
@@ -87,6 +88,7 @@
         for (int i = 0; i < numExprs; i++) {
             Mutable<ILogicalExpression> expRef = initExprs.get(i);
             AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) expRef.getValue();
+            SourceLocation aggFunSourceLoc = aggFun.getSourceLocation();
             IFunctionInfo fi1 = aggFun.getStepOneAggregate();
             // Clone the aggregate's args.
             List<Mutable<ILogicalExpression>> newArgs = new ArrayList<>(aggFun.getArguments().size());
@@ -98,10 +100,13 @@
             SimilarAggregatesInfo inf = new SimilarAggregatesInfo();
             LogicalVariable newAggVar = context.newVar();
             pushedVars.add(newAggVar);
-            inf.stepOneResult = new VariableReferenceExpression(newAggVar);
+            VariableReferenceExpression newAggVarRef = new VariableReferenceExpression(newAggVar);
+            newAggVarRef.setSourceLocation(aggFunSourceLoc);
+            inf.stepOneResult = newAggVarRef;
             inf.simAggs = new ArrayList<>();
             toReplaceSet.add(inf);
             AggregateFunctionCallExpression aggLocal = new AggregateFunctionCallExpression(fi1, false, newArgs);
+            aggLocal.setSourceLocation(aggFunSourceLoc);
             pushedExprs.add(new MutableObject<>(aggLocal));
             AggregateExprInfo aei = new AggregateExprInfo();
             aei.aggExprRef = expRef;
@@ -112,6 +117,7 @@
 
         if (!pushedVars.isEmpty()) {
             AggregateOperator pushedAgg = new AggregateOperator(pushedVars, pushedExprs);
+            pushedAgg.setSourceLocation(sourceLoc);
             pushedAgg.setExecutionMode(ExecutionMode.LOCAL);
             // If newGbyOp is null, then we optimizing an aggregate without group by.
             if (newGbyOp != null) {
@@ -133,6 +139,7 @@
 
                 // Hook up the nested aggregate op with the outer group by.
                 NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(newGbyOp));
+                nts.setSourceLocation(sourceLoc);
                 nts.setExecutionMode(ExecutionMode.LOCAL);
                 bottomRef.setValue(nts);
                 pushedAgg.getInputs().add(inputRef);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
index a921301..dc48d96 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractIntroduceGroupByCombinerRule extends AbstractIntroduceCombinerRule {
 
@@ -105,7 +106,9 @@
         for (LogicalVariable var : freeVars) {
             if (!propagatedVars.contains(var)) {
                 LogicalVariable newDecorVar = context.newVar();
-                newGbyOp.addDecorExpression(newDecorVar, new VariableReferenceExpression(var));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                varRef.setSourceLocation(gbyOp.getSourceLocation());
+                newGbyOp.addDecorExpression(newDecorVar, varRef);
                 VariableUtilities.substituteVariables(gbyOp.getNestedPlans().get(0).getRoots().get(0).getValue(), var,
                         newDecorVar, context);
             }
@@ -128,10 +131,12 @@
 
     private GroupByOperator opToPush(GroupByOperator gbyOp, BookkeepingInfo bi, IOptimizationContext context)
             throws AlgebricksException {
+        SourceLocation sourceLoc = gbyOp.getSourceLocation();
         // Hook up input to new group-by.
         Mutable<ILogicalOperator> opRef3 = gbyOp.getInputs().get(0);
         ILogicalOperator op3 = opRef3.getValue();
         GroupByOperator newGbyOp = new GroupByOperator();
+        newGbyOp.setSourceLocation(sourceLoc);
         newGbyOp.getInputs().add(new MutableObject<ILogicalOperator>(op3));
         // Copy annotations.
         Map<String, Object> annotations = newGbyOp.getAnnotations();
@@ -199,7 +204,9 @@
         // set the vars in the new op
         int n = newOpGbyList.size();
         for (int i = 0; i < n; i++) {
-            newGbyOp.addGbyExpression(replGbyList.get(i), new VariableReferenceExpression(newOpGbyList.get(i)));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(newOpGbyList.get(i));
+            varRef.setSourceLocation(sourceLoc);
+            newGbyOp.addGbyExpression(replGbyList.get(i), varRef);
             VariableUtilities.substituteVariables(gbyOp, newOpGbyList.get(i), replGbyList.get(i), false, context);
         }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index d975cce..ab665b3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -32,6 +32,7 @@
 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 BreakSelectIntoConjunctsRule implements IAlgebraicRewriteRule {
 
@@ -58,6 +59,8 @@
             return false;
         }
 
+        SourceLocation sourceLoc = select.getSourceLocation();
+
         Mutable<ILogicalOperator> childOfSelect = select.getInputs().get(0);
         boolean fst = true;
         ILogicalOperator botOp = select;
@@ -70,6 +73,7 @@
             } else {
                 SelectOperator newSelect = new SelectOperator(new MutableObject<ILogicalExpression>(e),
                         select.getRetainMissing(), select.getMissingPlaceholderVariable());
+                newSelect.setSourceLocation(sourceLoc);
                 List<Mutable<ILogicalOperator>> botInpList = botOp.getInputs();
                 botInpList.clear();
                 botInpList.add(new MutableObject<ILogicalOperator>(newSelect));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
index 0b94fdd..28139d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
@@ -83,6 +83,7 @@
         ntsToEtsInSubplan(subplan, context);
         cleanupJoins(subplan);
         InnerJoinOperator join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        join.setSourceLocation(op.getSourceLocation());
         join.getInputs().add(opRef3);
         opRef2.setValue(OperatorManipulationUtil.eliminateSingleSubplanOverEts(subplan));
         join.getInputs().add(new MutableObject<ILogicalOperator>(op));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
index fa35a98..54fe09d 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -117,6 +117,7 @@
 
         InnerJoinOperator product =
                 new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        product.setSourceLocation(op.getSourceLocation());
         // Outer branch.
         product.getInputs().add(new MutableObject<ILogicalOperator>(outerRoot));
         // Inner branch.
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
index ae52c35..d9849b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -79,6 +79,7 @@
             // Initialize the new conjuncts, if necessary.
             if (conj == null) {
                 conj = new ScalarFunctionCallExpression(andFn);
+                conj.setSourceLocation(firstSelect.getSourceLocation());
                 // Add the first select's condition.
                 conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
index 53548e4..1eaf9c7 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CopyLimitDownRule implements IAlgebraicRewriteRule {
 
@@ -84,9 +85,11 @@
             ILogicalOperator safeOp = safeOpRef.getValue();
             Mutable<ILogicalOperator> unsafeOpRef = safeOp.getInputs().get(0);
             ILogicalOperator unsafeOp = unsafeOpRef.getValue();
+            SourceLocation sourceLoc = limitOp.getSourceLocation();
             LimitOperator limitCloneOp = null;
             if (limitOp.getOffset().getValue() == null) {
                 limitCloneOp = new LimitOperator(limitOp.getMaxObjects().getValue(), false);
+                limitCloneOp.setSourceLocation(sourceLoc);
             } else {
                 // Need to add an offset to the given limit value
                 // since the original topmost limit will use the offset value.
@@ -98,7 +101,9 @@
                         new MutableObject<ILogicalExpression>(limitOp.getMaxObjects().getValue().cloneExpression()));
                 addArgs.add(new MutableObject<ILogicalExpression>(limitOp.getOffset().getValue().cloneExpression()));
                 ScalarFunctionCallExpression maxPlusOffset = new ScalarFunctionCallExpression(finfoAdd, addArgs);
+                maxPlusOffset.setSourceLocation(sourceLoc);
                 limitCloneOp = new LimitOperator(maxPlusOffset, false);
+                limitCloneOp.setSourceLocation(sourceLoc);
             }
             limitCloneOp.setPhysicalOperator(new StreamLimitPOperator());
             limitCloneOp.getInputs().add(new MutableObject<ILogicalOperator>(unsafeOp));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
index baad59b..096520e 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
@@ -141,6 +141,7 @@
             List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprs =
                     deepCopyOrderAndExpression(sourceOrderOp.getOrderExpressions());
             OrderOperator newOrderOp = new OrderOperator(orderExprs);
+            newOrderOp.setSourceLocation(sourceOrderOp.getSourceLocation());
             context.addToDontApplySet(this, newOrderOp);
             inputs.set(i, new MutableObject<ILogicalOperator>(newOrderOp));
             newOrderOp.getInputs().add(inputOpRef);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index f06933a..066b6c1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -89,6 +89,7 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 
 public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
@@ -526,16 +527,20 @@
     private Mutable<ILogicalOperator> enforceOrderProperties(List<LocalOrderProperty> oList,
             Mutable<ILogicalOperator> topOp, boolean isMicroOp, IOptimizationContext context)
             throws AlgebricksException {
+        SourceLocation sourceLoc = topOp.getValue().getSourceLocation();
         List<Pair<IOrder, Mutable<ILogicalExpression>>> oe = new LinkedList<>();
         for (LocalOrderProperty orderProperty : oList) {
             for (OrderColumn oc : orderProperty.getOrderColumns()) {
                 IOrder ordType = (oc.getOrder() == OrderKind.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
-                Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<>(ordType,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oc.getColumn())));
+                VariableReferenceExpression ocColumnRef = new VariableReferenceExpression(oc.getColumn());
+                ocColumnRef.setSourceLocation(sourceLoc);
+                Pair<IOrder, Mutable<ILogicalExpression>> pair =
+                        new Pair<>(ordType, new MutableObject<ILogicalExpression>(ocColumnRef));
                 oe.add(pair);
             }
         }
         OrderOperator oo = new OrderOperator(oe);
+        oo.setSourceLocation(sourceLoc);
         oo.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
         if (isMicroOp) {
             oo.setPhysicalOperator(new InMemoryStableSortPOperator());
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index b95d6e4..87053c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -48,6 +48,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 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;
 
 /**
  * Factors out common sub-expressions by assigning them to a variables, and replacing the common sub-expressions with references to those variables.
@@ -255,7 +256,10 @@
                         // Also just replace the expr if we are replacing common exprs from within the same operator.
                         if (liveVars.contains(exprEqClass.getVariable()) || !liveVars.containsAll(usedVars)
                                 || op == exprEqClass.getFirstOperator()) {
-                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(exprEqClass.getVariable());
+                            varRef.setSourceLocation(expr.getSourceLocation());
+                            exprRef.setValue(varRef);
                             // Do not descend into children since this expr has been completely replaced.
                             return true;
                         }
@@ -267,7 +271,10 @@
                         VariableUtilities.getLiveVariables(op, liveVars);
                         //rewrite only when the variable is live
                         if (liveVars.contains(exprEqClass.getVariable())) {
-                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(exprEqClass.getVariable());
+                            varRef.setSourceLocation(expr.getSourceLocation());
+                            exprRef.setValue(varRef);
                             // Do not descend into children since this expr has been completely replaced.
                             return true;
                         }
@@ -295,6 +302,7 @@
 
         private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr)
                 throws AlgebricksException {
+            SourceLocation sourceLoc = expr.getSourceLocation();
             AbstractLogicalOperator firstOp = (AbstractLogicalOperator) exprEqClass.getFirstOperator();
             Mutable<ILogicalExpression> firstExprRef = exprEqClass.getFirstExpression();
             if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN
@@ -313,6 +321,7 @@
                 // Place a Select operator beneath op that contains the enclosing expression.
                 SelectOperator selectOp =
                         new SelectOperator(new MutableObject<ILogicalExpression>(enclosingExpr), false, null);
+                selectOp.setSourceLocation(enclosingExpr.getSourceLocation());
                 selectOp.getInputs().add(new MutableObject<ILogicalOperator>(op.getInputs().get(0).getValue()));
                 op.getInputs().get(0).setValue(selectOp);
                 // Set firstOp to be the select below op, since we want to assign the common subexpr there.
@@ -324,12 +333,15 @@
             LogicalVariable newVar = context.newVar();
             AssignOperator newAssign = new AssignOperator(newVar,
                     new MutableObject<ILogicalExpression>(firstExprRef.getValue().cloneExpression()));
+            newAssign.setSourceLocation(sourceLoc);
             // Place assign below firstOp.
             newAssign.getInputs().add(new MutableObject<ILogicalOperator>(firstOp.getInputs().get(0).getValue()));
             newAssign.setExecutionMode(firstOp.getExecutionMode());
             firstOp.getInputs().get(0).setValue(newAssign);
             // Replace original expr with variable reference, and set var in expression equivalence class.
-            firstExprRef.setValue(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(sourceLoc);
+            firstExprRef.setValue(newVarRef);
             exprEqClass.setVariable(newVar);
             context.computeAndSetTypeEnvironmentForOperator(newAssign);
             context.computeAndSetTypeEnvironmentForOperator(firstOp);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 4c2d910..3335d71 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
 
@@ -162,7 +163,9 @@
                 continue;
             }
             candidate = group.get(0);
+            SourceLocation candidateSourceLoc = candidate.getValue().getSourceLocation();
             ReplicateOperator rop = new ReplicateOperator(group.size(), materializationFlags);
+            rop.setSourceLocation(candidateSourceLoc);
             rop.setPhysicalOperator(new ReplicatePOperator());
             Mutable<ILogicalOperator> ropRef = new MutableObject<ILogicalOperator>(rop);
             AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
@@ -204,7 +207,9 @@
             VariableUtilities.getLiveVariables(candidate.getValue(), liveVarsNew);
             ArrayList<Mutable<ILogicalExpression>> assignExprs = new ArrayList<Mutable<ILogicalExpression>>();
             for (LogicalVariable liveVar : liveVarsNew) {
-                assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(liveVar)));
+                VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+                liveVarRef.setSourceLocation(candidateSourceLoc);
+                assignExprs.add(new MutableObject<ILogicalExpression>(liveVarRef));
             }
             for (Mutable<ILogicalOperator> ref : group) {
                 if (ref.equals(candidate)) {
@@ -218,10 +223,14 @@
                     liveVars.add(variableMappingBack.get(liveVarsNew.get(i)));
                 }
 
+                SourceLocation refSourceLoc = ref.getValue().getSourceLocation();
+
                 AbstractLogicalOperator assignOperator = new AssignOperator(liveVars, assignExprs);
+                assignOperator.setSourceLocation(refSourceLoc);
                 assignOperator.setExecutionMode(rop.getExecutionMode());
                 assignOperator.setPhysicalOperator(new AssignPOperator());
                 AbstractLogicalOperator projectOperator = new ProjectOperator(liveVars);
+                projectOperator.setSourceLocation(refSourceLoc);
                 projectOperator.setPhysicalOperator(new StreamProjectPOperator());
                 projectOperator.setExecutionMode(rop.getExecutionMode());
                 AbstractLogicalOperator exchOp = new ExchangeOperator();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
index 198510a..d937eb1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 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;
 
 /**
  * Factors out function expressions from each comparison function or similarity function in join condition by
@@ -104,9 +105,11 @@
         } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi) || isComparisonFunction(fi)) {
             for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
                 if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    SourceLocation exprRefSourceLoc = exprRef.getValue().getSourceLocation();
                     LogicalVariable newVar = context.newVar();
                     AssignOperator newAssign = new AssignOperator(newVar,
                             new MutableObject<ILogicalExpression>(exprRef.getValue().cloneExpression()));
+                    newAssign.setSourceLocation(exprRefSourceLoc);
                     newAssign.setExecutionMode(joinOp.getExecutionMode());
 
                     // Place assign below joinOp.
@@ -137,7 +140,9 @@
 
                     if (modified) {
                         // Replace original expr with variable reference.
-                        exprRef.setValue(new VariableReferenceExpression(newVar));
+                        VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+                        newVarRef.setSourceLocation(exprRefSourceLoc);
+                        exprRef.setValue(newVarRef);
                         context.computeAndSetTypeEnvironmentForOperator(newAssign);
                         context.computeAndSetTypeEnvironmentForOperator(joinOp);
                     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
index 7a0012a..eb2bee6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
@@ -73,7 +73,9 @@
             ILogicalExpression expr = gbyPair.second.getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                gbyPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                gbyPair.second.setValue(vRef);
             }
         }
         return true;
@@ -88,7 +90,9 @@
             ILogicalExpression expr = decorPair.second.getValue();
             if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                decorPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                decorPair.second.setValue(vRef);
             }
         }
         return true;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
index 05cc7b6..5bbc80d 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
@@ -76,7 +76,9 @@
             exprs.add(exprRef);
 
             // Normalizes the decor entry -- expression be a variable reference
-            decorVarExpr.second = new MutableObject<>(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(expr.getSourceLocation());
+            decorVarExpr.second = new MutableObject<>(newVarRef);
         }
         if (!changed) {
             return false;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
index 2f28a84..d549b90 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FactorRedundantGroupAndDecorVarsRule implements IAlgebraicRewriteRule {
 
@@ -78,8 +79,11 @@
             LogicalVariable lhs = varRhsToLhs.get(v);
             if (lhs != null) {
                 if (p.first != null) {
-                    AssignOperator assign = new AssignOperator(p.first,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lhs)));
+                    VariableReferenceExpression lhsRef = new VariableReferenceExpression(lhs);
+                    SourceLocation sourceLoc = p.second.getValue().getSourceLocation();
+                    lhsRef.setSourceLocation(sourceLoc);
+                    AssignOperator assign = new AssignOperator(p.first, new MutableObject<ILogicalExpression>(lhsRef));
+                    assign.setSourceLocation(sourceLoc);
                     ILogicalOperator op = opRef.getValue();
                     assign.getInputs().add(new MutableObject<ILogicalOperator>(op));
                     opRef.setValue(assign);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
index d7090d2..e8f701c 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
@@ -195,7 +195,9 @@
                 LogicalVariable replacementVar = context.newVar();
                 assignVars.add(replacementVar);
                 assignExprs.add(new MutableObject<ILogicalExpression>(funcExpr));
-                funcExprRef.setValue(new VariableReferenceExpression(replacementVar));
+                VariableReferenceExpression replacementVarRef = new VariableReferenceExpression(replacementVar);
+                replacementVarRef.setSourceLocation(funcExpr.getSourceLocation());
+                funcExprRef.setValue(replacementVarRef);
                 funcIter.remove();
             }
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
index 4cd15a0..2ce0e63 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class EliminateSubplanRule implements IAlgebraicRewriteRule {
 
@@ -101,6 +102,7 @@
     private void elimSubplanOverEts(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx)
             throws AlgebricksException {
         SubplanOperator subplan = (SubplanOperator) opRef.getValue();
+        SourceLocation sourceLoc = subplan.getSourceLocation();
         for (ILogicalPlan p : subplan.getNestedPlans()) {
             for (Mutable<ILogicalOperator> r : p.getRoots()) {
                 OperatorManipulationUtil.ntsToEts(r, ctx);
@@ -117,6 +119,7 @@
                 } else {
                     InnerJoinOperator j =
                             new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+                    j.setSourceLocation(sourceLoc);
                     j.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
                     j.getInputs().add(r);
                     ctx.setOutputTypeEnvironment(j, j.computeOutputTypeEnvironment(ctx));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index 5118bf3..a988075 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -62,6 +62,7 @@
 import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * The rule searches for SUBPLAN operator with a optional PROJECT operator and
@@ -236,26 +237,35 @@
             default:
                 break;
         }
+
+        SourceLocation sourceLoc = subplan.getSourceLocation();
+
         if (testForNull == null) {
             testForNull = context.newVar();
             AssignOperator tmpAsgn =
                     new AssignOperator(testForNull, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+            tmpAsgn.setSourceLocation(sourceLoc);
             tmpAsgn.getInputs().add(new MutableObject<ILogicalOperator>(rightRef.getValue()));
             rightRef.setValue(tmpAsgn);
             context.computeAndSetTypeEnvironmentForOperator(tmpAsgn);
         }
 
         IFunctionInfo finfoEq = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.IS_MISSING);
-        ILogicalExpression isNullTest = new ScalarFunctionCallExpression(finfoEq,
+        ScalarFunctionCallExpression isNullTest = new ScalarFunctionCallExpression(finfoEq,
                 new MutableObject<ILogicalExpression>(new VariableReferenceExpression(testForNull)));
+        isNullTest.setSourceLocation(sourceLoc);
         IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
         ScalarFunctionCallExpression nonNullTest =
                 new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isNullTest));
+        nonNullTest.setSourceLocation(sourceLoc);
         SelectOperator selectNonNull =
                 new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
+        selectNonNull.setSourceLocation(sourceLoc);
         GroupByOperator g = new GroupByOperator();
+        g.setSourceLocation(sourceLoc);
         Mutable<ILogicalOperator> newSubplanRef = new MutableObject<ILogicalOperator>(subplan);
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(g));
+        nts.setSourceLocation(sourceLoc);
         opRef.setValue(g);
         selectNonNull.getInputs().add(new MutableObject<ILogicalOperator>(nts));
 
@@ -318,10 +328,12 @@
     private Map<LogicalVariable, LogicalVariable> buildVarExprList(Collection<LogicalVariable> vars,
             IOptimizationContext context, GroupByOperator g,
             List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+        SourceLocation sourceLoc = g.getSourceLocation();
         Map<LogicalVariable, LogicalVariable> m = new HashMap<LogicalVariable, LogicalVariable>();
         for (LogicalVariable ov : vars) {
             LogicalVariable newVar = context.newVar();
             ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+            ((VariableReferenceExpression) varExpr).setSourceLocation(sourceLoc);
             outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
                     new MutableObject<ILogicalExpression>(varExpr)));
             for (ILogicalPlan p : g.getNestedPlans()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
index f809e96..e37be1b 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
@@ -88,6 +88,7 @@
                 }
                 ntsRef.setValue(opUnder.getValue());
                 LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+                loj.setSourceLocation(join.getSourceLocation());
                 loj.getInputs().add(leftRef);
                 loj.getInputs().add(rightRef);
                 opRef.setValue(loj);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
index d9acf53..53b8eff 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
@@ -103,20 +104,24 @@
                 continue;
             }
 
+            SourceLocation sourceLoc = subplan.getSourceLocation();
+
             /**
              * Expends the input and roots into a DAG of nested loop joins.
              * Though joins should be left-outer joins, a left-outer join with condition TRUE is equivalent to an inner join.
              **/
             Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
             Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
-            ILogicalOperator join =
+            InnerJoinOperator join =
                     new InnerJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput), nestedRootRef);
+            join.setSourceLocation(sourceLoc);
 
             /** rewrite the nested tuple source to be empty tuple source */
             rewriteNestedTupleSource(nestedRootRef, context);
 
             for (int i = 1; i < nestedRoots.size(); i++) {
                 join = new InnerJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
+                join.setSourceLocation(sourceLoc);
             }
             op1.getInputs().get(index).setValue(join);
             context.computeAndSetTypeEnvironmentForOperator(join);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index f90de81..82b6f9c 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -22,7 +22,11 @@
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IPushRuntimeFactory extends Serializable {
     IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException;
+
+    default void setSourceLocation(SourceLocation sourceLoc) {
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
index 26cc5cb..c55d0f3 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
@@ -34,6 +34,10 @@
         this.value = value;
     }
 
+    public byte[] getValue() {
+        return value;
+    }
+
     @Override
     public String toString() {
         return "Constant";
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 0a578f6..b260a8a 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -150,8 +150,8 @@
                     ArrayTupleBuilder tb = outputWriter.getTupleBuilder();
                     byte[] data = tb.getByteArray();
                     if (data.length > memoryBudget) {
-                        throw HyracksDataException.create(ErrorCode.GROUP_BY_MEMORY_BUDGET_EXCEEDS, data.length,
-                                memoryBudget);
+                        throw HyracksDataException.create(ErrorCode.GROUP_BY_MEMORY_BUDGET_EXCEEDS, sourceLoc,
+                                data.length, memoryBudget);
                     }
                 }
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
index 75b2fb2..f057515 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -37,11 +37,11 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class NestedPlansRunningAggregatorFactory implements IAggregatorDescriptorFactory {
+public class NestedPlansRunningAggregatorFactory extends AbstractAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private final AlgebricksPipeline[] subplans;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 7b3fb46..525e452 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -19,11 +19,10 @@
 package org.apache.hyracks.algebricks.runtime.operators.base;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRuntimeFactory {
+public abstract class AbstractOneInputOneOutputRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java
new file mode 100644
index 0000000..89f3696
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.base;
+
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractPushRuntimeFactory implements IPushRuntimeFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    protected SourceLocation sourceLoc;
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
index f0e9406..1aceadc 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
@@ -21,11 +21,10 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class SinkRuntimeFactory implements IPushRuntimeFactory {
+public class SinkRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
index aefc99d..b1b652f 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
@@ -172,7 +172,7 @@
                         }
                     }
                 } catch (HyracksDataException e) {
-                    throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, tupleIndex);
+                    throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, tupleIndex);
                 }
             }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
index 67f4a77..7bd924d 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -19,15 +19,15 @@
 package org.apache.hyracks.algebricks.runtime.operators.std;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputSourcePushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 
-public class EmptyTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+public class EmptyTupleSourceRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index 8e64092..f94672d 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -21,12 +21,12 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+public class NestedTupleSourceRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
index 8a06ecf..7d6f851 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
@@ -21,12 +21,12 @@
 import org.apache.hyracks.algebricks.data.IAWriter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 
-public class PrinterRuntimeFactory implements IPushRuntimeFactory {
+public class PrinterRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
index d41b464..eae8178 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
@@ -28,12 +28,12 @@
 import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class SinkWriterRuntimeFactory implements IPushRuntimeFactory {
+public class SinkWriterRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
index 1706e59..e727551 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
@@ -24,13 +24,13 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class MicroUnionAllRuntimeFactory implements IPushRuntimeFactory {
+public class MicroUnionAllRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 23c41fe..ad2e77a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -39,8 +39,9 @@
         GET_JOB_STATUS,
         GET_JOB_INFO,
         START_JOB,
-        DISTRIBUTE_JOB,
-        DESTROY_JOB,
+        DEPLOY_JOB,
+        UNDEPLOY_JOB,
+        REDEPLOY_JOB,
         CANCEL_JOB,
         GET_DATASET_DIRECTORY_SERIVICE_INFO,
         GET_DATASET_RESULT_STATUS,
@@ -107,6 +108,32 @@
         }
     }
 
+    public static class redeployJobSpecFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final byte[] acggfBytes;
+
+        private final DeployedJobSpecId deployedJobSpecId;
+
+        public redeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) {
+            this.deployedJobSpecId = deployedJobSpecId;
+            this.acggfBytes = acggfBytes;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REDEPLOY_JOB;
+        }
+
+        public byte[] getACGGFBytes() {
+            return acggfBytes;
+        }
+
+        public DeployedJobSpecId getDeployedJobSpecId() {
+            return deployedJobSpecId;
+        }
+    }
+
     public static class DeployJobSpecFunction extends Function {
         private static final long serialVersionUID = 1L;
 
@@ -118,7 +145,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DISTRIBUTE_JOB;
+            return FunctionId.DEPLOY_JOB;
         }
 
         public byte[] getACGGFBytes() {
@@ -159,7 +186,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_JOB;
+            return FunctionId.UNDEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index eddcaa5..1ee9bd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -98,10 +98,17 @@
     }
 
     @Override
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception {
+        HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+                new HyracksClientInterfaceFunctions.redeployJobSpecFunction(deployedJobSpecId, acggfBytes);
+        rpci.call(ipcHandle, udjsf);
+    }
+
+    @Override
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
         HyracksClientInterfaceFunctions.UndeployJobSpecFunction sjf =
                 new HyracksClientInterfaceFunctions.UndeployJobSpecFunction(deployedJobSpecId);
-        return (DeployedJobSpecId) rpci.call(ipcHandle, sjf);
+        rpci.call(ipcHandle, sjf);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
index f676d27..eaec3c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
@@ -26,6 +26,11 @@
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpPut;
@@ -44,10 +49,16 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.topology.ClusterTopology;
+import org.apache.hyracks.api.util.InvokeUtil;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.ipc.api.RPCInterface;
 import org.apache.hyracks.ipc.impl.IPCSystem;
 import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.hyracks.util.InterruptibleAction;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 /**
  * Connection Class used by a Hyracks Client to interact with a Hyracks Cluster
@@ -56,6 +67,9 @@
  * @author vinayakb
  */
 public final class HyracksConnection implements IHyracksClientConnection {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private final String ccHost;
 
     private final int ccPort;
@@ -66,6 +80,15 @@
 
     private final ClusterControllerInfo ccInfo;
 
+    private volatile boolean running = false;
+
+    private volatile long reqId = 0L;
+
+    private final ExecutorService uninterruptibleExecutor =
+            Executors.newFixedThreadPool(2, r -> new Thread(r, "HyracksConnection Uninterrubtible thread: "));
+
+    private final BlockingQueue<UnInterruptibleRequest<?>> uninterruptibles = new ArrayBlockingQueue<>(1);
+
     /**
      * Constructor to create a connection to the Hyracks Cluster Controller.
      *
@@ -86,6 +109,8 @@
         hci = new HyracksClientInterfaceRemoteProxy(ipc.getReconnectingHandle(new InetSocketAddress(ccHost, ccPort)),
                 rpci);
         ccInfo = hci.getClusterControllerInfo();
+        uninterruptibleExecutor.execute(new UninterrubtileRequestHandler());
+        uninterruptibleExecutor.execute(new UninterrubtileHandlerWatcher());
     }
 
     @Override
@@ -95,7 +120,8 @@
 
     @Override
     public void cancelJob(JobId jobId) throws Exception {
-        hci.cancelJob(jobId);
+        CancelJobRequest request = new CancelJobRequest(jobId);
+        uninterruptiblySubmitAndExecute(request);
     }
 
     @Override
@@ -111,6 +137,13 @@
     }
 
     @Override
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception {
+        JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
+                new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
+        hci.redeployJobSpec(deployedJobSpecId, JavaSerializationUtils.serialize(jsacggf));
+    }
+
+    @Override
     public DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception {
         JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
                 new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
@@ -118,18 +151,19 @@
     }
 
     @Override
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
-        return hci.undeployJobSpec(deployedJobSpecId);
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+        hci.undeployJobSpec(deployedJobSpecId);
     }
 
     @Override
     public JobId startJob(DeployedJobSpecId deployedJobSpecId, Map<byte[], byte[]> jobParameters) throws Exception {
-        return hci.startJob(deployedJobSpecId, jobParameters);
+        StartDeployedJobRequest request = new StartDeployedJobRequest(deployedJobSpecId, jobParameters);
+        return interruptiblySubmitAndExecute(request);
     }
 
     @Override
     public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception {
-        return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
+        return startJob(null, acggf, jobFlags);
     }
 
     public DeployedJobSpecId deployJobSpec(IActivityClusterGraphGeneratorFactory acggf) throws Exception {
@@ -147,7 +181,7 @@
             hci.waitForCompletion(jobId);
         } catch (InterruptedException e) {
             // Cancels an on-going job if the current thread gets interrupted.
-            hci.cancelJob(jobId);
+            cancelJob(jobId);
             throw e;
         }
     }
@@ -225,7 +259,8 @@
     @Override
     public JobId startJob(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
             EnumSet<JobFlag> jobFlags) throws Exception {
-        return hci.startJob(deploymentId, JavaSerializationUtils.serialize(acggf), jobFlags);
+        StartJobRequest request = new StartJobRequest(deploymentId, acggf, jobFlags);
+        return interruptiblySubmitAndExecute(request);
     }
 
     @Override
@@ -262,4 +297,184 @@
     public boolean isConnected() {
         return hci.isConnected();
     }
+
+    private <T> T uninterruptiblySubmitAndExecute(UnInterruptibleRequest<T> request) throws Exception {
+        InvokeUtil.doUninterruptibly(() -> uninterruptibles.put(request));
+        return uninterruptiblyExecute(request);
+    }
+
+    private <T> T uninterruptiblyExecute(UnInterruptibleRequest<T> request) throws Exception {
+        InvokeUtil.doUninterruptibly(request);
+        return request.result();
+    }
+
+    private <T> T interruptiblySubmitAndExecute(UnInterruptibleRequest<T> request) throws Exception {
+        uninterruptibles.put(request);
+        return uninterruptiblyExecute(request);
+    }
+
+    private abstract class UnInterruptibleRequest<T> implements InterruptibleAction {
+        boolean completed = false;
+        boolean failed = false;
+        Throwable failure = null;
+        T response = null;
+
+        @SuppressWarnings("squid:S1181")
+        private final void handle() {
+            try {
+                response = doHandle();
+            } catch (Throwable th) {
+                failed = true;
+                failure = th;
+            } finally {
+                synchronized (this) {
+                    completed = true;
+                    notifyAll();
+                }
+            }
+        }
+
+        protected abstract T doHandle() throws Exception;
+
+        @Override
+        public final synchronized void run() throws InterruptedException {
+            while (!completed) {
+                wait();
+            }
+        }
+
+        public T result() throws Exception {
+            if (failed) {
+                if (failure instanceof Error) {
+                    throw (Error) failure;
+                }
+                throw (Exception) failure;
+            }
+            return response;
+        }
+    }
+
+    private class CancelJobRequest extends UnInterruptibleRequest<Void> {
+        final JobId jobId;
+
+        public CancelJobRequest(JobId jobId) {
+            this.jobId = jobId;
+        }
+
+        @Override
+        protected Void doHandle() throws Exception {
+            hci.cancelJob(jobId);
+            return null;
+        }
+
+        @Override
+        public String toString() {
+            return "CancelJobRequest: " + jobId.toString();
+        }
+
+    }
+
+    private class StartDeployedJobRequest extends UnInterruptibleRequest<JobId> {
+
+        private final DeployedJobSpecId deployedJobSpecId;
+        private final Map<byte[], byte[]> jobParameters;
+
+        public StartDeployedJobRequest(DeployedJobSpecId deployedJobSpecId, Map<byte[], byte[]> jobParameters) {
+            this.deployedJobSpecId = deployedJobSpecId;
+            this.jobParameters = jobParameters;
+        }
+
+        @Override
+        protected JobId doHandle() throws Exception {
+            return hci.startJob(deployedJobSpecId, jobParameters);
+        }
+
+    }
+
+    private class StartJobRequest extends UnInterruptibleRequest<JobId> {
+        private final DeploymentId deploymentId;
+        private final IActivityClusterGraphGeneratorFactory acggf;
+        private final EnumSet<JobFlag> jobFlags;
+
+        public StartJobRequest(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
+                EnumSet<JobFlag> jobFlags) {
+            this.deploymentId = deploymentId;
+            this.acggf = acggf;
+            this.jobFlags = jobFlags;
+        }
+
+        @Override
+        protected JobId doHandle() throws Exception {
+            if (deploymentId == null) {
+                return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
+            } else {
+                return hci.startJob(deploymentId, JavaSerializationUtils.serialize(acggf), jobFlags);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "StartJobRequest";
+        }
+
+    }
+
+    private class UninterrubtileRequestHandler implements Runnable {
+        @SuppressWarnings({ "squid:S2189", "squid:S2142" })
+        @Override
+        public void run() {
+            String nameBefore = Thread.currentThread().getName();
+            Thread.currentThread().setName(nameBefore + getClass().getSimpleName());
+            try {
+                while (true) {
+                    try {
+                        UnInterruptibleRequest<?> current = uninterruptibles.take();
+                        reqId++;
+                        running = true;
+                        current.handle();
+                    } catch (InterruptedException e) {
+                        LOGGER.log(Level.WARN, "Ignoring interrupt. This thread should never be interrupted.");
+                        continue;
+                    } finally {
+                        running = false;
+                    }
+                }
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+    }
+
+    public class UninterrubtileHandlerWatcher implements Runnable {
+        @Override
+        @SuppressWarnings({ "squid:S2189", "squid:S2142" })
+        public void run() {
+            String nameBefore = Thread.currentThread().getName();
+            Thread.currentThread().setName(nameBefore + getClass().getSimpleName());
+            try {
+                long currentReqId = 0L;
+                long currentTime = System.nanoTime();
+                while (true) {
+                    try {
+                        TimeUnit.MINUTES.sleep(1);
+                    } catch (InterruptedException e) {
+                        LOGGER.log(Level.WARN, "Ignoring interrupt. This thread should never be interrupted.");
+                        continue;
+                    }
+                    if (running) {
+                        if (reqId == currentReqId) {
+                            if (TimeUnit.NANOSECONDS.toMinutes(System.nanoTime() - currentTime) > 0) {
+                                ExitUtil.halt(ExitUtil.EC_FAILED_TO_PROCESS_UN_INTERRUPTIBLE_REQUEST);
+                            }
+                        } else {
+                            currentReqId = reqId;
+                            currentTime = System.nanoTime();
+                        }
+                    }
+                }
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 510a6b6..b3b7677 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -99,13 +99,24 @@
     DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception;
 
     /**
+     * Update the JobSpec for a deployed job.
+     *
+     * @param deployedJobSpecId
+     *            The id of the deployed job spec
+     * @param jobSpec
+     *            Job Specification
+     * @throws Exception
+     */
+    void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception;
+
+    /**
      * Remove the deployed Job Spec
      *
      * @param deployedJobSpecId
      *            The id of the deployed job spec
      * @throws Exception
      */
-    DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+    void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
 
     /**
      * Used to run a deployed Job Spec by id
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index f0c7872..6a75806 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -45,7 +45,9 @@
 
     public DeployedJobSpecId deployJobSpec(byte[] acggfBytes) throws Exception;
 
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception;
+
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
 
     public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
index 9148d6b..801dc37 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -20,11 +20,11 @@
 
 import java.io.Serializable;
 
-import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.constraints.IConstraintAcceptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Descriptor for operators in Hyracks.
@@ -37,7 +37,7 @@
      *
      * @return operator id
      */
-    public OperatorDescriptorId getOperatorId();
+    OperatorDescriptorId getOperatorId();
 
     /**
      * Sets the id of the operator.
@@ -51,21 +51,21 @@
      *
      * @return Number of inputs.
      */
-    public int getInputArity();
+    int getInputArity();
 
     /**
      * Returns the number of outputs out of this operator.
      *
      * @return Number of outputs.
      */
-    public int getOutputArity();
+    int getOutputArity();
 
     /**
      * Gets the output record descriptor
      *
      * @return Array of RecordDescriptor, one per output.
      */
-    public RecordDescriptor[] getOutputRecordDescriptors();
+    RecordDescriptor[] getOutputRecordDescriptors();
 
     /**
      * Contributes the activity graph that describes the behavior of this
@@ -74,7 +74,7 @@
      * @param builder
      *            - graph builder
      */
-    public void contributeActivities(IActivityGraphBuilder builder);
+    void contributeActivities(IActivityGraphBuilder builder);
 
     /**
      * Contributes any scheduling constraints imposed by this operator.
@@ -84,20 +84,30 @@
      * @param plan
      *            - Job Plan
      */
-    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext ccServiceCtx);
+    void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext ccServiceCtx);
 
     /**
      * Gets the display name.
      */
-    public String getDisplayName();
+    String getDisplayName();
 
     /**
      * Sets the display name.
      */
-    public void setDisplayName(String displayName);
+    void setDisplayName(String displayName);
+
+    /**
+     * Gets the source location.
+     */
+    SourceLocation getSourceLocation();
+
+    /**
+     * Sets the source location.
+     */
+    void setSourceLocation(SourceLocation sourceLoc);
 
     /**
      * Translates this operator descriptor to JSON.
      */
-    public ObjectNode toJSON();
+    ObjectNode toJSON();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
deleted file mode 100644
index c6d8f71..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.api.dataflow;
-
-public interface IOperatorNodePullable extends IOpenableDataReader<Object[]> {
-    public void setDataReader(int index, IOpenableDataWriter<Object[]> writer);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
deleted file mode 100644
index 4ec4c12..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.api.dataflow.value;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IHashFunction<T> {
-    public int hash(T o) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
deleted file mode 100644
index b22dbe7..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.api.dataflow.value;
-
-import java.io.Serializable;
-
-public interface IHashFunctionFactory<T> extends Serializable {
-    public IHashFunction<T> createHashFunction();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 51afac1..b6d7cc7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -121,7 +121,7 @@
     public static final int FOUND_MULTIPLE_TRANSACTIONS = 85;
     public static final int UNRECOGNIZED_INDEX_COMPONENT_FILE = 86;
     public static final int UNEQUAL_NUM_FILTERS_TREES = 87;
-    public static final int CANNOT_MODIFY_INDEX_DISK_IS_FULL = 88;
+    public static final int INDEX_NOT_MODIFIABLE = 88;
     public static final int GROUP_BY_MEMORY_BUDGET_EXCEEDS = 89;
     public static final int ILLEGAL_MEMORY_BUDGET = 90;
     public static final int TIMEOUT = 91;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index b9f84e8..54e4eaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -51,10 +51,21 @@
         return new HyracksDataException(cause);
     }
 
+    public static HyracksDataException create(int code, SourceLocation sourceLoc, Serializable... params) {
+        return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), null, sourceLoc,
+                params);
+    }
+
     public static HyracksDataException create(int code, Serializable... params) {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
     }
 
+    public static HyracksDataException create(int code, Throwable cause, SourceLocation sourceLoc,
+            Serializable... params) {
+        return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, sourceLoc,
+                params);
+    }
+
     public static HyracksDataException create(int code, Throwable cause, Serializable... params) {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
@@ -111,6 +122,11 @@
         super(component, errorCode, message, cause, null, params);
     }
 
+    public HyracksDataException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, Serializable... params) {
+        super(component, errorCode, message, cause, sourceLoc, null, params);
+    }
+
     public static HyracksDataException create(HyracksDataException e, String nodeId) {
         return new HyracksDataException(e.getComponent(), e.getErrorCode(), e.getMessage(), e.getCause(), nodeId,
                 e.getStackTrace(), e.getParams());
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index 210779e..428e643 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -23,7 +23,7 @@
 
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
-public class HyracksException extends IOException {
+public class HyracksException extends IOException implements IFormattedException {
     private static final long serialVersionUID = 1L;
 
     public static final int UNKNOWN = 0;
@@ -31,6 +31,7 @@
     private final int errorCode;
     private final Serializable[] params;
     private final String nodeId;
+    private final SourceLocation sourceLoc;
     private transient volatile String msgCache;
 
     public static HyracksException create(Throwable cause) {
@@ -59,9 +60,10 @@
         return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
 
-    public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
-            Serializable... params) {
+    public HyracksException(String component, int errorCode, String message, Throwable cause, SourceLocation sourceLoc,
+            String nodeId, Serializable... params) {
         super(message, cause);
+        this.sourceLoc = sourceLoc;
         this.component = component;
         this.errorCode = errorCode;
         this.nodeId = nodeId;
@@ -73,7 +75,7 @@
      */
     @Deprecated
     public HyracksException(String message) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
     }
 
     /**
@@ -81,7 +83,7 @@
      */
     @Deprecated
     protected HyracksException(Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, String.valueOf(cause), cause, null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, String.valueOf(cause), cause, (Serializable[]) null);
     }
 
     /**
@@ -112,10 +114,17 @@
         this(component, errorCode, message, cause, null, params);
     }
 
+    public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
+            Serializable... params) {
+        this(component, errorCode, message, cause, null, nodeId, params);
+    }
+
+    @Override
     public String getComponent() {
         return component;
     }
 
+    @Override
     public int getErrorCode() {
         return errorCode;
     }
@@ -128,10 +137,14 @@
         return nodeId;
     }
 
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
     @Override
     public String getMessage() {
         if (msgCache == null) {
-            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params);
+            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params);
         }
         return msgCache;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
new file mode 100644
index 0000000..994915f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.exceptions;
+
+public interface IFormattedException {
+
+    /**
+     * Gets the component of this {@link IFormattedException}
+     *
+     * @return the exception component
+     */
+    String getComponent();
+
+    /**
+     * Gets the error code of this {@link IFormattedException}
+     *
+     * @return the error code
+     */
+    int getErrorCode();
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
new file mode 100644
index 0000000..fd3992c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.exceptions;
+
+import java.io.Serializable;
+
+public final class SourceLocation implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int line;
+
+    private final int column;
+
+    public SourceLocation(int line, int column) {
+        this.line = line;
+        this.column = column;
+    }
+
+    public int getLine() {
+        return line;
+    }
+
+    public int getColumn() {
+        return column;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index 26ce2c1..56dfe3f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -27,6 +27,7 @@
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -83,12 +84,15 @@
      * @param errorCode
      *            the error code itself
      * @param message
-     *            the user provided error message (a format string as specified in {@link java.util.Formatter})
+     *            the user provided error message (a format string as specified in {@link Formatter})
+     * @param sourceLoc
+     *            the source location where the error originated
      * @param params
-     *            an array of objects taht will be provided to the {@link java.util.Formatter}
+     *            an array of objects taht will be provided to the {@link Formatter}
      * @return the formatted string
      */
-    public static String formatMessage(String component, int errorCode, String message, Serializable... params) {
+    public static String formatMessage(String component, int errorCode, String message, SourceLocation sourceLoc,
+            Serializable... params) {
         try (Formatter fmt = new Formatter()) {
             if (!NONE.equals(component)) {
                 fmt.format("%1$s%2$04d: ", component, errorCode);
@@ -98,6 +102,10 @@
                 return message;
             }
             fmt.format(message == null ? "null" : message, (Object[]) params);
+            if (sourceLoc != null) {
+                fmt.out().append(" (in line ").append(String.valueOf(sourceLoc.getLine())).append(", at column ")
+                        .append(String.valueOf(sourceLoc.getColumn())).append(')');
+            }
             return fmt.out().toString();
         } catch (Exception e) {
             // Do not throw further exceptions during exception processing.
@@ -105,5 +113,4 @@
             return e.getMessage();
         }
     }
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 9302f46..bba0de7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -119,4 +119,15 @@
         stackThrowable.setStackTrace(thread.getStackTrace());
         return stackThrowable;
     }
+
+    public static Throwable getRootCause(Throwable e) {
+        Throwable current = e;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            cause = current.getCause();
+        }
+        return current;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
index b2dd680..9413d1b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
@@ -26,8 +26,12 @@
 import java.util.function.BooleanSupplier;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ComputingAction;
+import org.apache.hyracks.util.IDelay;
 import org.apache.hyracks.util.IOInterruptibleAction;
+import org.apache.hyracks.util.IRetryPolicy;
 import org.apache.hyracks.util.InterruptibleAction;
+import org.apache.hyracks.util.Span;
 import org.apache.hyracks.util.ThrowingAction;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -217,7 +221,7 @@
     }
 
     /**
-     * Runs the supplied action, after suspending any pending interruption.  An error will be logged if
+     * Runs the supplied action, after suspending any pending interruption. An error will be logged if
      * the action is itself interrupted.
      */
     public static void runUninterruptible(ThrowingAction action) throws Exception {
@@ -251,4 +255,30 @@
             }
         }
     }
+
+    public static <T> T retryUntilSuccessOrExhausted(Span span, ComputingAction<T> action, IRetryPolicy policy,
+            IDelay delay) throws HyracksDataException {
+        Throwable failure;
+        int attempt = 0;
+        do {
+            attempt++;
+            try {
+                return action.compute();
+            } catch (Throwable th) {
+                failure = th;
+                if (!policy.retry(th)) {
+                    break;
+                }
+                try {
+                    LOGGER.log(Level.WARN, "Failure executing action {} for the {} time", action, attempt, failure);
+                    span.sleep(delay.calculate(attempt), TimeUnit.MILLISECONDS);
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
+        } while (!span.elapsed());
+        LOGGER.log(Level.WARN, "Final Failure executing action {} after {} attempts", action, attempt, failure);
+        throw HyracksDataException.create(failure);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
index ccf0163..85019c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
@@ -65,20 +65,20 @@
         }
     }
 
-    public void stop() throws HyracksDataException, InterruptedException {
+    public void stop() throws HyracksDataException {
         stopped = true;
         executorThread.interrupt();
-        executorThread.join(1000);
+        InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         int attempt = 0;
         while (executorThread.isAlive()) {
             attempt++;
-            LOGGER.log(Level.WARN,
-                    "Failed to stop event processor after " + attempt + " attempts. Interrupted exception swallowed?");
+            LOGGER.log(Level.WARN, "Failed to stop event processor after {} attempts. Interrupted exception swallowed?",
+                    attempt, ExceptionUtils.fromThreadStack(executorThread));
             if (attempt == 10) {
                 throw HyracksDataException.create(ErrorCode.FAILED_TO_SHUTDOWN_EVENT_PROCESSOR, name);
             }
             executorThread.interrupt();
-            executorThread.join(1000);
+            InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 452d379..ef07038 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -104,7 +104,7 @@
 85 = Found more than one transaction file in %1$s
 86 = Found an unrecognized index file %1$s
 87 = Unequal number of trees and filters found in %1$s
-88 = Cannot modify index (Disk is full)
+88 = Cannot modify index (%1$s)
 89 = The byte size of a single group (%1$s bytes) exceeds the budget for a group by operator (%2$s bytes)
 90 = Memory budget for the %1$s operator (%2$s bytes) is lower than the minimum (%3$s bytes)
 91 = Operation timed out
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
index 23a1caa..c8c1790 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
@@ -36,7 +36,7 @@
     @Test
     public void returnedMessageWithNoComponentTest() {
         HyracksDataException cause = new HyracksDataException(ErrorMessageUtil.NONE, ErrorCode.ERROR_PROCESSING_TUPLE,
-                ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), null, null, 2);
+                ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), 2);
         HyracksDataException causeWithNodeId = HyracksDataException.create(cause, "nc1");
         Assert.assertEquals(cause.getMessage(), causeWithNodeId.getMessage());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e46aa7f..a669402 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -85,13 +85,19 @@
                 ccs.getWorkQueue().schedule(new GetJobInfoWork(ccs.getJobManager(), gjif.getJobId(),
                         new IPCResponder<JobInfo>(handle, mid)));
                 break;
-            case DISTRIBUTE_JOB:
+            case DEPLOY_JOB:
                 HyracksClientInterfaceFunctions.DeployJobSpecFunction djf =
                         (HyracksClientInterfaceFunctions.DeployJobSpecFunction) fn;
                 ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, djf.getACGGFBytes(),
-                        deployedJobSpecIdFactory.create(), new IPCResponder<>(handle, mid)));
+                        deployedJobSpecIdFactory.create(), false, new IPCResponder<>(handle, mid)));
                 break;
-            case DESTROY_JOB:
+            case REDEPLOY_JOB:
+                HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+                        (HyracksClientInterfaceFunctions.redeployJobSpecFunction) fn;
+                ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, udjsf.getACGGFBytes(),
+                        udjsf.getDeployedJobSpecId(), true, new IPCResponder<>(handle, mid)));
+                break;
+            case UNDEPLOY_JOB:
                 HyracksClientInterfaceFunctions.UndeployJobSpecFunction dsjf =
                         (HyracksClientInterfaceFunctions.UndeployJobSpecFunction) fn;
                 ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
index 0e22c25..041e224 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
@@ -40,9 +40,6 @@
     public void addDeployedJobSpecDescriptor(DeployedJobSpecId deployedJobSpecId,
             ActivityClusterGraph activityClusterGraph, JobSpecification jobSpecification,
             Set<Constraint> activityClusterGraphConstraints) throws HyracksException {
-        if (deployedJobSpecDescriptorMap.get(deployedJobSpecId) != null) {
-            throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
-        }
         DeployedJobSpecDescriptor descriptor =
                 new DeployedJobSpecDescriptor(activityClusterGraph, jobSpecification, activityClusterGraphConstraints);
         deployedJobSpecDescriptorMap.put(deployedJobSpecId, descriptor);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
index b44a6bb8..2b03324 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -105,9 +105,7 @@
             failNode(nodeId);
         }
         try {
-            // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
-            IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(ncState.getNodeController().getAddress());
-            ncIPCHandle.send(-1, new AbortCCJobsFunction(ccConfig.getCcId()), null);
+            ncState.getNodeController().abortJobs(ccs.getCcId());
         } catch (IPCException e) {
             throw HyracksDataException.create(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
index c51f3c5..60c88c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
@@ -39,20 +39,24 @@
     private final byte[] acggfBytes;
     private final DeployedJobSpecId deployedJobSpecId;
     private final IResultCallback<DeployedJobSpecId> callback;
+    private final boolean upsert;
 
     public DeployJobSpecWork(ClusterControllerService ccs, byte[] acggfBytes, DeployedJobSpecId deployedJobSpecId,
-            IResultCallback<DeployedJobSpecId> callback) {
+            boolean upsert, IResultCallback<DeployedJobSpecId> callback) {
         this.deployedJobSpecId = deployedJobSpecId;
         this.ccs = ccs;
         this.acggfBytes = acggfBytes;
         this.callback = callback;
+        this.upsert = upsert;
     }
 
     @Override
     protected void doRun() throws Exception {
         try {
             final CCServiceContext ccServiceCtx = ccs.getContext();
-            ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+            if (!upsert) {
+                ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+            }
             IActivityClusterGraphGeneratorFactory acggf =
                     (IActivityClusterGraphGeneratorFactory) DeploymentUtils.deserialize(acggfBytes, null, ccServiceCtx);
             IActivityClusterGraphGenerator acgg =
@@ -65,7 +69,7 @@
 
             INodeManager nodeManager = ccs.getNodeManager();
             for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
-                node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes);
+                node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes, upsert);
             }
             callback.setValue(deployedJobSpecId);
         } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
index 517f56f..aa7dca1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
@@ -19,7 +19,8 @@
 
 package org.apache.hyracks.control.cc.work;
 
-import java.io.File;
+import static org.apache.hyracks.control.common.utils.ConfigurationUtil.toPathElements;
+
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
@@ -31,19 +32,22 @@
 import java.util.Date;
 import java.util.List;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.config.Section;
 import org.apache.hyracks.control.cc.NodeControllerState;
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.config.ConfigUtils;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.hyracks.util.PidHelper;
 import org.apache.hyracks.control.common.work.IPCResponder;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.hyracks.util.MXHelper;
+import org.apache.hyracks.util.PidHelper;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class GetNodeDetailsJSONWork extends SynchronizableWork {
+
     private static final Section[] CC_SECTIONS = { Section.CC, Section.COMMON };
     private static final Section[] NC_SECTIONS = { Section.NC, Section.COMMON };
 
@@ -114,9 +118,9 @@
             o.put("vm_name", runtimeMXBean.getVmName());
             o.put("vm_version", runtimeMXBean.getVmVersion());
             o.put("vm_vendor", runtimeMXBean.getVmVendor());
-            o.putPOJO("classpath", runtimeMXBean.getClassPath().split(File.pathSeparator));
-            o.putPOJO("library_path", runtimeMXBean.getLibraryPath().split(File.pathSeparator));
-            o.putPOJO("boot_classpath", runtimeMXBean.getBootClassPath().split(File.pathSeparator));
+            o.putPOJO("classpath", toPathElements(runtimeMXBean.getClassPath()));
+            o.putPOJO("library_path", toPathElements(runtimeMXBean.getLibraryPath()));
+            o.putPOJO("boot_classpath", toPathElements(MXHelper.getBootClassPath()));
             o.putPOJO("input_arguments", runtimeMXBean.getInputArguments());
             o.putPOJO("system_properties", runtimeMXBean.getSystemProperties());
             o.put("pid", PidHelper.getPid());
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index de7d941..00693df 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -28,11 +28,8 @@
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
-import org.apache.hyracks.control.common.ipc.CCNCFunctions;
 import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
-import org.apache.hyracks.ipc.api.IIPCHandle;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -52,36 +49,33 @@
     @Override
     protected void doRun() throws Exception {
         String id = reg.getNodeId();
-        // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
-        IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
-        CCNCFunctions.NodeRegistrationResult result;
-        Map<IOption, Object> ncConfiguration = new HashMap<>();
+        LOGGER.warn("Registering node: {}", id);
+        NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
+                ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
+        INodeManager nodeManager = ccs.getNodeManager();
         try {
-            LOGGER.log(Level.WARN, "Registering INodeController: id = " + id);
-            NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
-                    ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
             NodeControllerState state = new NodeControllerState(nc, reg);
-            INodeManager nodeManager = ccs.getNodeManager();
             nodeManager.addNode(id, state);
             IApplicationConfig cfg = state.getNCConfig().getConfigManager().getNodeEffectiveConfig(id);
+            final Map<IOption, Object> ncConfiguration = new HashMap<>();
             for (IOption option : cfg.getOptions()) {
                 ncConfiguration.put(option, cfg.get(option));
             }
-            LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
+            LOGGER.warn("Registered node: {}", id);
             NodeParameters params = new NodeParameters();
             params.setClusterControllerInfo(ccs.getClusterControllerInfo());
             params.setDistributedState(ccs.getContext().getDistributedState());
             params.setHeartbeatPeriod(ccs.getCCConfig().getHeartbeatPeriodMillis());
             params.setProfileDumpPeriod(ccs.getCCConfig().getProfileDumpPeriod());
             params.setRegistrationId(registrationId);
-            result = new CCNCFunctions.NodeRegistrationResult(params, null);
+            LOGGER.warn("sending registration response to node {}", id);
+            nc.sendRegistrationResult(params, null);
+            LOGGER.warn("notifying node {} joined", id);
+            ccs.getContext().notifyNodeJoin(id, ncConfiguration);
         } catch (Exception e) {
-            LOGGER.log(Level.WARN, "Node registration failed", e);
-            result = new CCNCFunctions.NodeRegistrationResult(null, e);
+            LOGGER.error("Node {} registration failed", id, e);
+            nodeManager.removeNode(id);
+            nc.sendRegistrationResult(null, e);
         }
-        LOGGER.warn("sending registration response to node");
-        ncIPCHandle.send(-1, result, null);
-        LOGGER.warn("notifying node join");
-        ccs.getContext().notifyNodeJoin(id, ncConfiguration);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
index 143c8c1..69b55ed 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
@@ -28,10 +28,10 @@
 public class UndeployJobSpecWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final DeployedJobSpecId deployedJobSpecId;
-    private final IResultCallback<DeployedJobSpecId> callback;
+    private final IResultCallback<Void> callback;
 
     public UndeployJobSpecWork(ClusterControllerService ccs, DeployedJobSpecId deployedJobSpecId,
-            IResultCallback<DeployedJobSpecId> callback) {
+            IResultCallback<Void> callback) {
         this.deployedJobSpecId = deployedJobSpecId;
         this.ccs = ccs;
         this.callback = callback;
@@ -45,7 +45,7 @@
             for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
                 node.getNodeController().undeployJobSpec(deployedJobSpecId);
             }
-            callback.setValue(deployedJobSpecId);
+            callback.setValue(null);
         } catch (Exception e) {
             callback.setException(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
index 9959e34..cc2ed46 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -134,7 +134,6 @@
         IPCSystem ipcSystem = Mockito.mock(IPCSystem.class);
         IIPCHandle ipcHandle = Mockito.mock(IIPCHandle.class);
         Mockito.when(ccs.getClusterIPC()).thenReturn(ipcSystem);
-        Mockito.when(ipcSystem.getHandle(Mockito.any())).thenReturn(ipcHandle);
         Mockito.when(ipcSystem.getHandle(Mockito.any(), Mockito.anyInt())).thenReturn(ipcHandle);
         Mockito.when(ccs.getExecutor()).thenReturn(Executors.newCachedThreadPool());
         return ccs;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
index 8dfc729..ddab504 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
@@ -18,16 +18,15 @@
  */
 package org.apache.hyracks.control.common;
 
+import static org.apache.hyracks.control.common.utils.ConfigurationUtil.toPathElements;
 import static org.apache.hyracks.util.JSONUtil.put;
 
-import java.io.File;
 import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.resource.NodeCapacity;
@@ -301,9 +300,9 @@
             put(o, "vm-name", vmName);
             put(o, "vm-version", vmVersion);
             put(o, "vm-vendor", vmVendor);
-            put(o, "classpath", StringUtils.split(classpath, File.pathSeparatorChar));
-            put(o, "library-path", StringUtils.split(libraryPath, File.pathSeparatorChar));
-            put(o, "boot-classpath", StringUtils.split(bootClasspath, File.pathSeparatorChar));
+            put(o, "classpath", toPathElements(classpath));
+            put(o, "library-path", toPathElements(libraryPath));
+            put(o, "boot-classpath", toPathElements(bootClasspath));
             put(o, "input-arguments", inputArguments);
             put(o, "input-arguments", inputArguments);
             put(o, "system-properties", systemProperties);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
index 92764a7..fa835f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
@@ -24,6 +24,7 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
@@ -33,7 +34,9 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
+import org.apache.hyracks.ipc.exceptions.IPCException;
 
 public interface INodeController {
     void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
@@ -51,7 +54,8 @@
 
     void undeployBinary(DeploymentId deploymentId) throws Exception;
 
-    void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception;
+    void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean checkForDuplicate)
+            throws Exception;
 
     void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
 
@@ -62,4 +66,22 @@
     void sendApplicationMessageToNC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception;
 
     void takeThreadDump(String requestId) throws Exception;
+
+    /**
+     * Sends a request to this {@link INodeController} to abort all jobs
+     * started by cluster controller with id {@code ccId}
+     *
+     * @param ccId
+     * @throws IPCException
+     */
+    void abortJobs(CcId ccId) throws IPCException;
+
+    /**
+     * Sends node registration result to this {@link INodeController}.
+     *
+     * @param parameters
+     * @param regFailure
+     * @throws IPCException
+     */
+    void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
index 8dd95e1..b49dc20 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
@@ -103,7 +103,6 @@
 
     public ConfigManager(String[] args) {
         this.args = args;
-        checkJavaVersion();
         for (Section section : Section.values()) {
             allSections.add(section.sectionName());
         }
@@ -113,14 +112,6 @@
         addConfigurator(ConfiguratorMetric.APPLY_DEFAULTS, this::applyDefaults);
     }
 
-    static void checkJavaVersion() {
-        final String javaVersion = System.getProperty("java.version");
-        LOGGER.info("Found JRE version " + javaVersion);
-        if (!javaVersion.startsWith("1.8")) {
-            throw new IllegalStateException("JRE version 1.8 is required");
-        }
-    }
-
     @Override
     public void addConfigurator(int metric, IConfigurator configurator) {
         configurators.computeIfAbsent(metric, metric1 -> new ArrayList<>()).add(configurator);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
index 76f5ad8..c2f2e1a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hyracks.control.common.controllers;
 
+import static org.apache.hyracks.util.MXHelper.osMXBean;
+import static org.apache.hyracks.util.MXHelper.runtimeMXBean;
+
 import java.io.Serializable;
 import java.net.InetSocketAddress;
 import java.util.List;
@@ -26,6 +29,8 @@
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.job.resource.NodeCapacity;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatSchema;
+import org.apache.hyracks.util.MXHelper;
+import org.apache.hyracks.util.PidHelper;
 
 public final class NodeRegistration implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -73,31 +78,28 @@
     private final NodeCapacity capacity;
 
     public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
-            NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors, String vmName,
-            String vmVersion, String vmVendor, String classpath, String libraryPath, String bootClasspath,
-            List<String> inputArguments, Map<String, String> systemProperties, HeartbeatSchema hbSchema,
-            NetworkAddress messagingPort, NodeCapacity capacity, int pid) {
+            NetworkAddress datasetPort, HeartbeatSchema hbSchema, NetworkAddress messagingPort, NodeCapacity capacity) {
         this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
         this.dataPort = dataPort;
         this.datasetPort = datasetPort;
-        this.osName = osName;
-        this.arch = arch;
-        this.osVersion = osVersion;
-        this.nProcessors = nProcessors;
-        this.vmName = vmName;
-        this.vmVersion = vmVersion;
-        this.vmVendor = vmVendor;
-        this.classpath = classpath;
-        this.libraryPath = libraryPath;
-        this.bootClasspath = bootClasspath;
-        this.inputArguments = inputArguments;
-        this.systemProperties = systemProperties;
         this.hbSchema = hbSchema;
         this.messagingPort = messagingPort;
         this.capacity = capacity;
-        this.pid = pid;
+        this.osName = osMXBean.getName();
+        this.arch = osMXBean.getArch();
+        this.osVersion = osMXBean.getVersion();
+        this.nProcessors = osMXBean.getAvailableProcessors();
+        this.vmName = runtimeMXBean.getVmName();
+        this.vmVersion = runtimeMXBean.getVmVersion();
+        this.vmVendor = runtimeMXBean.getVmVendor();
+        this.classpath = runtimeMXBean.getClassPath();
+        this.libraryPath = runtimeMXBean.getLibraryPath();
+        this.bootClasspath = MXHelper.getBootClassPath();
+        this.inputArguments = runtimeMXBean.getInputArguments();
+        this.systemProperties = runtimeMXBean.getSystemProperties();
+        this.pid = PidHelper.getPid();
     }
 
     public InetSocketAddress getNodeControllerAddress() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
index 8e02936..dea5198 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
@@ -103,8 +103,8 @@
         SHUTDOWN_REQUEST,
         SHUTDOWN_RESPONSE,
 
-        DISTRIBUTE_JOB,
-        DESTROY_JOB,
+        DEPLOY_JOB,
+        UNDEPLOY_JOB,
         DEPLOYED_JOB_FAILURE,
 
         STATE_DUMP_REQUEST,
@@ -713,15 +713,18 @@
 
         private final byte[] acgBytes;
 
-        public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, CcId ccId) {
+        private final boolean upsert;
+
+        public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, boolean upsert, CcId ccId) {
             super(ccId);
             this.deployedJobSpecId = deployedJobSpecId;
             this.acgBytes = acgBytes;
+            this.upsert = upsert;
         }
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DISTRIBUTE_JOB;
+            return FunctionId.DEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
@@ -731,6 +734,10 @@
         public byte[] getacgBytes() {
             return acgBytes;
         }
+
+        public boolean getUpsert() {
+            return upsert;
+        }
     }
 
     public static class UndeployJobSpecFunction extends CCIdentifiedFunction {
@@ -745,7 +752,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_JOB;
+            return FunctionId.UNDEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index b78e53f..8242bdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.common.base.INodeController;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.AbortTasksFunction;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.CleanupJobletFunction;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.DeployBinaryFunction;
@@ -50,6 +51,7 @@
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.UndeployJobSpecFunction;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
 import org.apache.hyracks.ipc.api.IIPCHandle;
+import org.apache.hyracks.ipc.exceptions.IPCException;
 
 public class NodeControllerRemoteProxy implements INodeController {
     private final CcId ccId;
@@ -101,8 +103,8 @@
     }
 
     @Override
-    public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception {
-        DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, ccId);
+    public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean upsert) throws Exception {
+        DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, upsert, ccId);
         ipcHandle.send(-1, fn, null);
     }
 
@@ -136,6 +138,16 @@
         ipcHandle.send(-1, fn, null);
     }
 
+    @Override
+    public void abortJobs(CcId ccId) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.AbortCCJobsFunction(ccId), null);
+    }
+
+    @Override
+    public void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.NodeRegistrationResult(parameters, regFailure), null);
+    }
+
     public InetSocketAddress getAddress() {
         return ipcHandle.getRemoteAddress();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ConfigurationUtil.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ConfigurationUtil.java
index d7cef4e..45d4787 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ConfigurationUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ConfigurationUtil.java
@@ -18,10 +18,18 @@
  */
 package org.apache.hyracks.control.common.utils;
 
+import java.io.File;
+
+import org.apache.commons.lang3.StringUtils;
+
 public class ConfigurationUtil {
     public static final String JAVA_IO_TMPDIR = "java.io.tmpdir";
+    private static final String[] EMPTY_STRING_ARR = new String[0];
 
     private ConfigurationUtil() {
     }
 
+    public static String[] toPathElements(String path) {
+        return path == null ? EMPTY_STRING_ARR : StringUtils.split(path, File.pathSeparatorChar);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
index 735f7cf..08cd5d8 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
@@ -107,13 +107,13 @@
                 ncs.getWorkQueue().schedule(new UnDeployBinaryWork(ncs, ndbf.getDeploymentId(), ndbf.getCcId()));
                 return;
 
-            case DISTRIBUTE_JOB:
+            case DEPLOY_JOB:
                 CCNCFunctions.DeployJobSpecFunction djf = (CCNCFunctions.DeployJobSpecFunction) fn;
-                ncs.getWorkQueue().schedule(
-                        new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(), djf.getCcId()));
+                ncs.getWorkQueue().schedule(new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(),
+                        djf.getUpsert(), djf.getCcId()));
                 return;
 
-            case DESTROY_JOB:
+            case UNDEPLOY_JOB:
                 CCNCFunctions.UndeployJobSpecFunction dsjf = (CCNCFunctions.UndeployJobSpecFunction) fn;
                 ncs.getWorkQueue().schedule(new UndeployJobSpecWork(ncs, dsjf.getDeployedJobSpecId(), dsjf.getCcId()));
                 return;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index 6a7d645..804bacd 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -20,8 +20,6 @@
 
 import static org.apache.hyracks.util.MXHelper.gcMXBeans;
 import static org.apache.hyracks.util.MXHelper.memoryMXBean;
-import static org.apache.hyracks.util.MXHelper.osMXBean;
-import static org.apache.hyracks.util.MXHelper.runtimeMXBean;
 
 import java.io.File;
 import java.io.IOException;
@@ -94,7 +92,6 @@
 import org.apache.hyracks.net.protocols.muxdemux.FullFrameChannelInterfaceFactory;
 import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.MaintainedThreadNameExecutorService;
-import org.apache.hyracks.util.PidHelper;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.TraceUtils;
 import org.apache.hyracks.util.trace.Tracer;
@@ -330,12 +327,8 @@
         NetworkAddress netAddress = netManager.getPublicNetworkAddress();
         NetworkAddress messagingAddress =
                 messagingNetManager != null ? messagingNetManager.getPublicNetworkAddress() : null;
-        nodeRegistration = new NodeRegistration(ncAddress, id, ncConfig, netAddress, datasetAddress, osMXBean.getName(),
-                osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(),
-                runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(),
-                runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
-                runtimeMXBean.getSystemProperties(), hbSchema, messagingAddress, application.getCapacity(),
-                PidHelper.getPid());
+        nodeRegistration = new NodeRegistration(ncAddress, id, ncConfig, netAddress, datasetAddress, hbSchema,
+                messagingAddress, application.getCapacity());
 
         ncData = new NodeControllerData(nodeRegistration);
     }
@@ -470,7 +463,6 @@
     }
 
     private ConcurrentHashMap<CcId, Serializable> getDistributedState() {
-        //noinspection unchecked
         return (ConcurrentHashMap<CcId, Serializable>) serviceCtx.getDistributedState();
     }
 
@@ -566,9 +558,6 @@
 
     public void storeActivityClusterGraph(DeployedJobSpecId deployedJobSpecId, ActivityClusterGraph acg)
             throws HyracksException {
-        if (deployedJobSpecActivityClusterGraphMap.get(deployedJobSpecId.getId()) != null) {
-            throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
-        }
         deployedJobSpecActivityClusterGraphMap.put(deployedJobSpecId.getId(), acg);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
index b832b20..6b35912 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
@@ -61,9 +61,9 @@
 
     private FileReference fileRef;
 
-    private IFileHandle writeFileHandle;
+    private IFileHandle fileHandle;
 
-    private IFileHandle readFileHandle;
+    private volatile int referenceCount = 0;
 
     private long size;
 
@@ -86,12 +86,13 @@
         localPageList = new ArrayList<>();
 
         fileRef = null;
-        writeFileHandle = null;
+        fileHandle = null;
     }
 
     public synchronized void open() {
         size = 0;
         persistentSize = 0;
+        referenceCount = 0;
     }
 
     public synchronized void close() {
@@ -112,25 +113,29 @@
     }
 
     private void closeWriteFileHandle() {
-        if (writeFileHandle != null) {
+        if (fileHandle != null) {
+            doCloseFileHandle();
+        }
+    }
+
+    private void doCloseFileHandle() {
+        if (--referenceCount == 0) {
+            // close the file if there is no more reference
             try {
-                ioManager.close(writeFileHandle);
+                ioManager.close(fileHandle);
             } catch (IOException e) {
                 // Since file handle could not be closed, just ignore.
             }
-            writeFileHandle = null;
+            fileHandle = null;
         }
     }
 
     public synchronized void write(ByteBuffer buffer) throws HyracksDataException {
         if (fileRef == null) {
-            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
-            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
-            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_SYNC);
+            initWriteFileHandle();
         }
 
-        size += ioManager.syncWrite(writeFileHandle, size, buffer);
+        size += ioManager.syncWrite(fileHandle, size, buffer);
         notifyAll();
     }
 
@@ -165,9 +170,8 @@
     }
 
     public synchronized void readClose() throws HyracksDataException {
-        if (readFileHandle != null) {
-            ioManager.close(readFileHandle);
-            readFileHandle = null;
+        if (fileHandle != null) {
+            doCloseFileHandle();
         }
     }
 
@@ -185,51 +189,49 @@
             return readSize;
         }
 
-        if (readFileHandle == null) {
+        if (fileHandle == null) {
             initReadFileHandle();
         }
-        readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+        readSize = ioManager.syncRead(fileHandle, offset, buffer);
 
         return readSize;
     }
 
-    public long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
+    public synchronized long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
             throws HyracksDataException {
         long readSize = 0;
-        synchronized (this) {
-            while (offset >= size && !eos.get() && !failed.get()) {
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                    throw HyracksDataException.create(e);
-                }
+        while (offset >= size && !eos.get() && !failed.get()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw HyracksDataException.create(e);
             }
+        }
 
-            if ((offset >= size && eos.get()) || failed.get()) {
+        if ((offset >= size && eos.get()) || failed.get()) {
+            return readSize;
+        }
+
+        if (offset < persistentSize) {
+            if (fileHandle == null) {
+                initReadFileHandle();
+            }
+            readSize = ioManager.syncRead(fileHandle, offset, buffer);
+            if (readSize < 0) {
+                throw new HyracksDataException("Premature end of file");
+            }
+        }
+
+        if (readSize < buffer.capacity()) {
+            long localPageOffset = offset - persistentSize;
+            int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
+            int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
+            Page page = getPage(localPageIndex);
+            if (page == null) {
                 return readSize;
             }
-
-            if (offset < persistentSize) {
-                if (readFileHandle == null) {
-                    initReadFileHandle();
-                }
-                readSize = ioManager.syncRead(readFileHandle, offset, buffer);
-                if (readSize < 0) {
-                    throw new HyracksDataException("Premature end of file");
-                }
-            }
-
-            if (readSize < buffer.capacity()) {
-                long localPageOffset = offset - persistentSize;
-                int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
-                int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
-                Page page = getPage(localPageIndex);
-                if (page == null) {
-                    return readSize;
-                }
-                readSize += buffer.remaining();
-                buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
-            }
+            readSize += buffer.remaining();
+            buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
         }
         datasetMemoryManager.pageReferenced(resultSetPartitionId);
         return readSize;
@@ -245,21 +247,17 @@
 
         // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
         if (page == null) {
-            ioManager.close(writeFileHandle);
+            ioManager.close(fileHandle);
             return null;
         }
 
         page.getBuffer().flip();
 
         if (fileRef == null) {
-            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
-            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
-            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-            notifyAll();
+            initWriteFileHandle();
         }
 
-        long delta = ioManager.syncWrite(writeFileHandle, persistentSize, page.getBuffer());
+        long delta = ioManager.syncWrite(fileHandle, persistentSize, page.getBuffer());
         persistentSize += delta;
         return page;
     }
@@ -325,8 +323,23 @@
         return page;
     }
 
+    private void initWriteFileHandle() throws HyracksDataException {
+        if (fileHandle == null) {
+            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
+            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
+            fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+            if (referenceCount != 0) {
+                throw new IllegalStateException("Illegal reference count " + referenceCount);
+            }
+            referenceCount = 1;
+            notifyAll(); // NOSONAR: always called from a synchronized block
+        }
+    }
+
     private void initReadFileHandle() throws HyracksDataException {
         while (fileRef == null && !failed.get()) {
+            // wait for writer to create the file
             try {
                 wait();
             } catch (InterruptedException e) {
@@ -336,9 +349,12 @@
         if (failed.get()) {
             return;
         }
-
-        readFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
-                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        if (fileHandle == null) {
+            // fileHandle has been closed by the writer, create it again
+            fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        }
+        referenceCount++;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 7de716a..1f845bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -71,7 +71,7 @@
      * Mutables
      */
     private int workspaceIndex;
-    private IFileDeviceResolver deviceComputer;
+    private final IFileDeviceResolver deviceComputer;
 
     public IOManager(List<IODeviceHandle> devices, IFileDeviceResolver deviceComputer) throws HyracksDataException {
         this.ioDevices = Collections.unmodifiableList(devices);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index 33d1d60..6d4f173 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -46,7 +46,7 @@
     public void run() {
         NCServiceContext ctx = ncs.getContext();
         try {
-            IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);;
+            IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);
             if (ctx.getMessageBroker() != null) {
                 ctx.getMessageBroker().receivedMessage(data, nodeId);
             } else {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
index 92612dd..bcdb97f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.control.nc.NodeControllerService;
 
 /**
- * pre-distribute a job that can be executed later
+ * Deploy a job that can be executed later
  *
  */
 public class DeployJobSpecWork extends AbstractWork {
@@ -37,19 +37,23 @@
     private final byte[] acgBytes;
     private final CcId ccId;
     private final DeployedJobSpecId deployedJobSpecId;
+    private final boolean upsert;
 
     public DeployJobSpecWork(NodeControllerService ncs, DeployedJobSpecId deployedJobSpecId, byte[] acgBytes,
-            CcId ccId) {
+            boolean upsert, CcId ccId) {
         this.ncs = ncs;
         this.deployedJobSpecId = deployedJobSpecId;
         this.acgBytes = acgBytes;
         this.ccId = ccId;
+        this.upsert = upsert;
     }
 
     @Override
     public void run() {
         try {
-            ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+            if (!upsert) {
+                ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+            }
             ActivityClusterGraph acg =
                     (ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, null, ncs.getContext());
             ncs.storeActivityClusterGraph(deployedJobSpecId, acg);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
index 5964c04..0f36c80 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
@@ -66,11 +66,11 @@
                 LOGGER.error("{} tasks associated with CC {} failed to complete after {}ms. Giving up",
                         runningTasks.size(), ccId, TIMEOUT);
                 logPendingTasks();
-                ExitUtil.halt(ExitUtil.NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
+                ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
             }
         } catch (Throwable th) {
             LOGGER.error("Failed to abort all previous tasks associated with CC {}", ccId, th);
-            ExitUtil.halt(ExitUtil.NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
+            ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 8fdcd83..376307d 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -587,6 +587,31 @@
         builder.finish();
     }
 
+    /**
+     * Generates a reversed string from an input source string
+     *
+     * @param srcPtr
+     *            , the input source string.
+     * @param builder
+     *            , a builder for the resulting string.
+     * @param out
+     *            , the storage for a result string.
+     * @throws IOException
+     */
+    public static void reverse(UTF8StringPointable srcPtr, UTF8StringBuilder builder, GrowableArray out)
+            throws IOException {
+        builder.reset(out, srcPtr.getUTF8Length());
+        int srcStart = srcPtr.getCharStartOffset();
+        int srcEnd = srcPtr.getStartOffset() + srcPtr.getLength() - 1;
+        for (int cursorIndex = srcEnd; cursorIndex >= srcStart; cursorIndex--) {
+            if (UTF8StringUtil.isCharStart(srcPtr.bytes, cursorIndex)) {
+                int charSize = UTF8StringUtil.charSize(srcPtr.bytes, cursorIndex);
+                builder.appendUtf8StringPointable(srcPtr, cursorIndex, charSize);
+            }
+        }
+        builder.finish();
+    }
+
     public boolean findAndReplace(UTF8StringPointable searchPtr, UTF8StringPointable replacePtr, int replaceLimit,
             UTF8StringBuilder builder, GrowableArray out) throws IOException {
         return findAndReplace(this, searchPtr, replacePtr, replaceLimit, builder, out);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
index 08ed922..49b5309 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
@@ -164,4 +164,30 @@
             tupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
         }
     }
+
+    public static boolean equalTuples(ITupleReference tuple1, ITupleReference tuple2, int numCmpFields) {
+        for (int i = 0; i < numCmpFields; i++) {
+            if (!equalFields(tuple1, tuple2, i)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static boolean equalFields(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
+        return equalFields(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx), tuple1.getFieldLength(fIdx),
+                tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx), tuple2.getFieldLength(fIdx));
+    }
+
+    public static boolean equalFields(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
+        if (aLength != bLength) {
+            return false;
+        }
+        for (int i = 0; i < aLength; i++) {
+            if (a[aOffset + i] != b[bOffset + i]) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index c6512929..205d1ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -43,6 +44,8 @@
 
     protected String displayName;
 
+    protected SourceLocation sourceLoc;
+
     public AbstractOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity) {
         odId = spec.createOperatorDescriptorId(this);
         this.inputArity = inputArity;
@@ -87,6 +90,16 @@
     }
 
     @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
     public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext serviceCtx) {
         // do nothing
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
index d546e5e..4426acb 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
@@ -23,7 +23,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public abstract class AbstractAccumulatingAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+public abstract class AbstractAccumulatingAggregatorDescriptorFactory extends AbstractAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..12ad3ec
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.dataflow.std.group;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+    private static final long serialVersionUID = 1L;
+
+    protected SourceLocation sourceLoc;
+
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
deleted file mode 100644
index fd16c52..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.dataflow.std.group;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.dataflow.IDataReader;
-import org.apache.hyracks.api.dataflow.IDataWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IGroupAggregator extends Serializable {
-    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
-
-    public void close() throws Exception;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index bee0590..dc250e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -91,13 +91,12 @@
      * @param hashFunctionFactories
      * @param comparatorFactories
      * @param recordDescriptor
-     * @throws HyracksDataException
      */
     public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
-            IMissingWriterFactory[] nullWriterFactories1) throws HyracksDataException {
+            IMissingWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.memsize = memsize;
         this.inputsize0 = inputsize0;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 9eeb363..b68e64e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -144,7 +144,7 @@
             IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, ITuplePairComparatorFactory tupPaircomparatorFactory01,
             ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory,
-            boolean isLeftOuter, IMissingWriterFactory[] nonMatchWriterFactories) throws HyracksDataException {
+            boolean isLeftOuter, IMissingWriterFactory[] nonMatchWriterFactories) {
         super(spec, 2, 1);
         this.memSizeInFrames = memSizeInFrames;
         this.inputsize0 = inputsize0;
@@ -165,8 +165,7 @@
             int inputsize0, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, ITuplePairComparatorFactory tupPaircomparatorFactory01,
-            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory)
-            throws HyracksDataException {
+            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory) {
         this(spec, memSizeInFrames, inputsize0, factor, keys0, keys1, hashFunctionGeneratorFactories,
                 comparatorFactories, recordDescriptor, tupPaircomparatorFactory01, tupPaircomparatorFactory10,
                 predEvaluatorFactory, false, null);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
index 40b6b27..d8f4064 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
@@ -199,6 +199,16 @@
         for (int i = 0; i < 100; i++) {
             hcc.startJob(distributedId2, new HashMap<>());
         }
+
+        //Change the second job into the first job and see whether it runs
+        hcc.redeployJobSpec(distributedId2, spec1);
+        JobId jobRunId4 = hcc.startJob(distributedId2, new HashMap<>());
+        hcc.waitForCompletion(jobRunId4);
+
+        //Run it one more time
+        JobId jobRunId5 = hcc.startJob(distributedId2, new HashMap<>());
+        hcc.waitForCompletion(jobRunId5);
+
     }
 
     @AfterClass
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
index 6f4726e..99112c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
@@ -140,7 +140,8 @@
           <version>1.0.4</version>
           <type>jar</type>
           <scope>test</scope>
-        </dependency>      </dependencies>
+        </dependency>
+      </dependencies>
     </profile>
 
     <profile>
@@ -170,23 +171,13 @@
         <dependency>
           <groupId>org.apache.hyracks</groupId>
           <artifactId>hyracks-control-common</artifactId>
-          <version>0.3.4-SNAPSHOT</version>
+          <version>${project.version}</version>
           <scope>test</scope>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
           <version>${hadoop.version}</version>
-          <exclusions>
-            <exclusion>
-              <groupId>javax.servlet.jsp</groupId>
-              <artifactId>jsp-api</artifactId>
-            </exclusion>
-            <exclusion>
-              <groupId>javax.servlet</groupId>
-              <artifactId>servlet-api</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
index d4f1b3d..891cc2a 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
@@ -105,7 +105,7 @@
                 response.beforeFlush();
                 DefaultHttpContent content = new DefaultHttpContent(buffer);
                 ctx.writeAndFlush(content, ctx.channel().voidPromise());
-                // The responisbility of releasing the buffer is now with the netty pipeline since it is forwarded
+                // The responsibility of releasing the buffer is now with the netty pipeline since it is forwarded
                 // within the http content. We must nullify buffer before we allocate the next one to avoid
                 // releasing the buffer twice in case the allocation call fails.
                 buffer = null;
@@ -128,13 +128,13 @@
                 wait();
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
-                LOGGER.log(Level.WARN, "Interupted while waiting for channel to be writable", e);
+                LOGGER.log(Level.WARN, "Interrupted while waiting for channel to be writable", e);
                 throw new IOException(e);
             }
         }
     }
 
-    public synchronized void resume() {
+    public synchronized void channelWritabilityChanged() {
         notifyAll();
     }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
index 5a43d25..cd746b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
@@ -185,11 +185,11 @@
 
     @Override
     public void notifyChannelWritable() {
-        outputStream.resume();
+        outputStream.channelWritabilityChanged();
     }
 
     @Override
     public void notifyChannelInactive() {
-        outputStream.resume();
+        outputStream.channelWritabilityChanged();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
index 41e0088..bbe64ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
@@ -39,10 +39,12 @@
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.FixedRecvByteBufAllocator;
 import io.netty.channel.WriteBufferWaterMark;
+import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.logging.LogLevel;
@@ -242,7 +244,7 @@
                 .childOption(ChannelOption.AUTO_READ, Boolean.FALSE)
                 .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
                 .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, WRITE_BUFFER_WATER_MARK)
-                .handler(new LoggingHandler(LogLevel.DEBUG)).childHandler(new HttpServerInitializer(this));
+                .handler(new LoggingHandler(LogLevel.DEBUG)).childHandler(getChannelInitializer());
         Channel newChannel = b.bind(port).sync().channel();
         newChannel.closeFuture().addListener(f -> {
             // This listener is invoked from within a netty IO thread. Hence, we can never block it
@@ -381,6 +383,10 @@
         return new HttpServerHandler<>(this, chunkSize);
     }
 
+    protected ChannelInitializer<SocketChannel> getChannelInitializer() {
+        return new HttpServerInitializer(this);
+    }
+
     public ThreadPoolExecutor getExecutor(HttpRequestHandler handler) {
         return executor;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index ffa4d4b..8d6dfbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -19,36 +19,23 @@
 package org.apache.hyracks.http.server.utils;
 
 import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryPoolMXBean;
-import java.lang.management.MemoryType;
-import java.lang.reflect.Field;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.BaseRequest;
 import org.apache.hyracks.http.server.FormUrlEncodedRequest;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
-import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.codec.http.HttpHeaderNames;
 import io.netty.handler.codec.http.HttpRequest;
-import io.netty.util.internal.PlatformDependent;
 
 public class HttpUtil {
-
-    private static final Logger LOGGER = LogManager.getLogger();
     private static final Pattern PARENT_DIR = Pattern.compile("/[^./]+/\\.\\./");
-    private static long maxMemUsage = 0L;
 
     private HttpUtil() {
     }
@@ -63,6 +50,7 @@
     public static class ContentType {
         public static final String APPLICATION_ADM = "application/x-adm";
         public static final String APPLICATION_JSON = "application/json";
+        public static final String JSON = "json";
         public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded";
         public static final String CSV = "text/csv";
         public static final String IMG_PNG = "image/png";
@@ -166,53 +154,4 @@
         return clusterURL;
     }
 
-    @SuppressWarnings("restriction")
-    public static synchronized void printMemUsage() {
-        StringBuilder report = new StringBuilder();
-        report.append("sun.misc.VM.maxDirectMemory: ");
-        report.append(sun.misc.VM.maxDirectMemory());
-        report.append('\n');
-        report.append("sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getMemoryUsed(): ");
-        report.append(sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getMemoryUsed());
-        report.append('\n');
-        report.append("sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getTotalCapacity(): ");
-        report.append(sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getTotalCapacity());
-        report.append('\n');
-        report.append("ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage(): ");
-        report.append(ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage());
-        report.append('\n');
-        report.append("---------------------------- Beans ----------------------------");
-        report.append('\n');
-        List<MemoryPoolMXBean> memPoolBeans = ManagementFactory.getMemoryPoolMXBeans();
-        for (MemoryPoolMXBean bean : memPoolBeans) {
-            if (bean.isValid() && bean.getType() == MemoryType.NON_HEAP) {
-                report.append(bean.getName());
-                report.append(": ");
-                report.append(bean.getUsage());
-                report.append('\n');
-            }
-        }
-        report.append("---------------------------- Netty ----------------------------");
-        report.append('\n');
-        try {
-            Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
-            field.setAccessible(true);
-            AtomicLong usedDirectMemory = (AtomicLong) field.get(null);
-            long used = usedDirectMemory.get();
-            report.append("Current PlatformDependent.DIRECT_MEMORY_COUNTER: ");
-            report.append(used);
-            report.append('\n');
-            report.append("Maximum PlatformDependent.DIRECT_MEMORY_COUNTER: ");
-            maxMemUsage = Math.max(maxMemUsage, used);
-            report.append(maxMemUsage);
-            report.append('\n');
-            report.append('\n');
-        } catch (Throwable th) { // NOSONAR
-            LOGGER.log(Level.WARN, "Failed to access PlatformDependent.DIRECT_MEMORY_COUNTER", th);
-            return;
-        }
-        report.append("--------------- PooledByteBufAllocator.DEFAULT ----------------");
-        report.append(PooledByteBufAllocator.DEFAULT.dumpStats());
-        LOGGER.log(Level.INFO, report.toString());
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/HttpTestUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/HttpTestUtil.java
new file mode 100644
index 0000000..16efc07
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/HttpTestUtil.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryPoolMXBean;
+import java.lang.management.MemoryType;
+import java.lang.reflect.Field;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.internal.PlatformDependent;
+
+public class HttpTestUtil {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static long maxMemUsage = 0L;
+
+    public static synchronized void printMemUsage() {
+        StringBuilder report = new StringBuilder();
+        /* TODO: unsupported APIs after java 8
+        report.append("sun.misc.VM.maxDirectMemory: ");
+        report.append(sun.misc.VM.maxDirectMemory());
+        report.append('\n');
+        report.append("sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getMemoryUsed(): ");
+        report.append(sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getMemoryUsed());
+        report.append('\n');
+        report.append("sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getTotalCapacity(): ");
+        report.append(sun.misc.SharedSecrets.getJavaNioAccess().getDirectBufferPool().getTotalCapacity());
+        */
+        report.append('\n');
+        report.append("ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage(): ");
+        report.append(ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage());
+        report.append('\n');
+        report.append("---------------------------- Beans ----------------------------");
+        report.append('\n');
+        List<MemoryPoolMXBean> memPoolBeans = ManagementFactory.getMemoryPoolMXBeans();
+        for (MemoryPoolMXBean bean : memPoolBeans) {
+            if (bean.isValid() && bean.getType() == MemoryType.NON_HEAP) {
+                report.append(bean.getName());
+                report.append(": ");
+                report.append(bean.getUsage());
+                report.append('\n');
+            }
+        }
+        report.append("---------------------------- Netty ----------------------------");
+        report.append('\n');
+        try {
+            Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
+            field.setAccessible(true);
+            AtomicLong usedDirectMemory = (AtomicLong) field.get(null);
+            long used = usedDirectMemory.get();
+            report.append("Current PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+            report.append(used);
+            report.append('\n');
+            report.append("Maximum PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+            maxMemUsage = Math.max(maxMemUsage, used);
+            report.append(maxMemUsage);
+            report.append('\n');
+            report.append('\n');
+        } catch (Throwable th) { // NOSONAR
+            LOGGER.log(Level.WARN, "Failed to access PlatformDependent.DIRECT_MEMORY_COUNTER", th);
+            return;
+        }
+        report.append("--------------- PooledByteBufAllocator.DEFAULT ----------------");
+        report.append(PooledByteBufAllocator.DEFAULT.dumpStats());
+        LOGGER.log(Level.INFO, report.toString());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/servlet/ChattyServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/servlet/ChattyServlet.java
index e6aedb9..54bb756 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/servlet/ChattyServlet.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/servlet/ChattyServlet.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.hyracks.http.HttpTestUtil;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
@@ -58,6 +59,6 @@
         for (int i = 0; i < 100; i++) {
             response.outputStream().write(bytes);
         }
-        HttpUtil.printMemUsage();
+        HttpTestUtil.printMemUsage();
     }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
index 71e84e5..b39a141 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
@@ -32,10 +32,10 @@
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hyracks.http.HttpTestUtil;
 import org.apache.hyracks.http.server.HttpServer;
 import org.apache.hyracks.http.server.InterruptOnCloseHandler;
 import org.apache.hyracks.http.server.WebManager;
-import org.apache.hyracks.http.server.utils.HttpUtil;
 import org.apache.hyracks.http.servlet.ChattyServlet;
 import org.apache.hyracks.http.servlet.SleepyServlet;
 import org.apache.logging.log4j.Level;
@@ -147,7 +147,7 @@
         try {
             try {
                 for (int i = 0; i < numPatches; i++) {
-                    HttpUtil.printMemUsage();
+                    HttpTestUtil.printMemUsage();
                     request(numRequests);
                     for (Future<Void> f : FUTURES) {
                         f.get();
@@ -155,7 +155,7 @@
                     FUTURES.clear();
                 }
             } finally {
-                HttpUtil.printMemUsage();
+                HttpTestUtil.printMemUsage();
                 servlet.wakeUp();
                 for (Future<Void> f : stuck) {
                     f.get();
@@ -165,7 +165,7 @@
             System.err.println("Number of rejections: " + UNAVAILABLE_COUNT.get());
             System.err.println("Number of exceptions: " + EXCEPTION_COUNT.get());
             webMgr.stop();
-            HttpUtil.printMemUsage();
+            HttpTestUtil.printMemUsage();
         }
     }
 
@@ -180,7 +180,7 @@
         int numRequests = 48;
         int numExecutors = 24;
         int serverQueueSize = 24;
-        HttpUtil.printMemUsage();
+        HttpTestUtil.printMemUsage();
         WebManager webMgr = new WebManager();
         HttpServer server =
                 new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
@@ -197,9 +197,9 @@
             Assert.assertEquals(0, UNAVAILABLE_COUNT.get());
             Assert.assertEquals(0, OTHER_COUNT.get());
         } finally {
-            HttpUtil.printMemUsage();
+            HttpTestUtil.printMemUsage();
             webMgr.stop();
-            HttpUtil.printMemUsage();
+            HttpTestUtil.printMemUsage();
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
index 7f59db1..1f9ec37 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
@@ -41,10 +41,6 @@
   </properties>
   <dependencies>
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
deleted file mode 100644
index 4328fc3..0000000
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.ipc.api;
-
-public interface IResponseCallback {
-    public void callback(IIPCHandle handle, Object response, Exception exception);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
index 912c267..39a29af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
@@ -23,6 +23,5 @@
     CONNECT_SENT,
     CONNECT_RECEIVED,
     CONNECTED,
-    CONNECT_FAILED,
     CLOSED,
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index 3e6c64b..040fe03 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -18,12 +18,13 @@
  */
 package org.apache.hyracks.ipc.impl;
 
+import static org.apache.hyracks.util.ExitUtil.EC_IMMEDIATE_HALT;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channel;
-import java.nio.channels.ClosedChannelException;
 import java.nio.channels.SelectableChannel;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.Selector;
@@ -39,7 +40,7 @@
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.NetworkUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -100,7 +101,7 @@
 
     void stop() {
         stopped = true;
-        IOUtils.closeQuietly(serverSocketChannel);
+        NetworkUtil.closeQuietly(serverSocketChannel);
         networkThread.selector.wakeup();
     }
 
@@ -121,8 +122,10 @@
                 return handle;
             }
             if (maxRetries < 0 || retries++ < maxRetries) {
-                LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
-                        : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
+                            : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+                }
                 Thread.sleep(delay);
                 delay = Math.min(MAX_RETRY_DELAY_MILLIS, (int) (delay * 1.5));
             } else {
@@ -144,24 +147,6 @@
         networkThread.selector.wakeup();
     }
 
-    private synchronized void collectOutstandingWork() {
-        if (!pendingConnections.isEmpty()) {
-            moveAll(pendingConnections, workingPendingConnections);
-        }
-        if (!sendList.isEmpty()) {
-            moveAll(sendList, workingSendList);
-        }
-    }
-
-    private Message createInitialReqMessage(IPCHandle handle) {
-        Message msg = new Message(handle);
-        msg.setMessageId(system.createMessageId());
-        msg.setRequestMessageId(-1);
-        msg.setFlag(Message.INITIAL_REQ);
-        msg.setPayload(address);
-        return msg;
-    }
-
     private Message createInitialAckMessage(IPCHandle handle, Message req) {
         Message msg = new Message(handle);
         msg.setMessageId(system.createMessageId());
@@ -177,16 +162,18 @@
 
     private class NetworkThread extends Thread {
         private final Selector selector;
-
         private final Set<SocketChannel> openChannels = new HashSet<>();
+        private final BitSet unsentMessagesBitmap = new BitSet();
+        private final List<Message> tempUnsentMessages = new ArrayList<>();
 
-        public NetworkThread() {
+        NetworkThread() {
             super("IPC Network Listener Thread [" + address + "]");
             setDaemon(true);
             try {
                 selector = Selector.open();
+                serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
             } catch (IOException e) {
-                throw new RuntimeException(e);
+                throw new IllegalStateException(e);
             }
         }
 
@@ -200,105 +187,19 @@
         }
 
         private void doRun() {
-            try {
-                serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
-            } catch (ClosedChannelException e) {
-                throw new RuntimeException(e);
-            }
-            BitSet unsentMessagesBitmap = new BitSet();
-            List<Message> tempUnsentMessages = new ArrayList<>();
             int failingLoops = 0;
             while (!stopped) {
                 try {
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("Starting Select");
-                    }
                     int n = selector.select();
                     collectOutstandingWork();
                     if (!workingPendingConnections.isEmpty()) {
-                        for (IPCHandle handle : workingPendingConnections) {
-                            SocketChannel channel = SocketChannel.open();
-                            register(channel);
-                            SelectionKey cKey;
-                            if (channel.connect(handle.getRemoteAddress())) {
-                                cKey = channel.register(selector, SelectionKey.OP_READ);
-                                handle.setState(HandleState.CONNECT_SENT);
-                                IPCConnectionManager.this.write(createInitialReqMessage(handle));
-                            } else {
-                                cKey = channel.register(selector, SelectionKey.OP_CONNECT);
-                            }
-                            handle.setKey(cKey);
-                            cKey.attach(handle);
-                        }
-                        workingPendingConnections.clear();
+                        establishPendingConnections();
                     }
                     if (!workingSendList.isEmpty()) {
-                        unsentMessagesBitmap.clear();
-                        int len = workingSendList.size();
-                        for (int i = 0; i < len; ++i) {
-                            Message msg = workingSendList.get(i);
-                            LOGGER.debug(() -> "Processing send of message: " + msg);
-                            IPCHandle handle = msg.getIPCHandle();
-                            if (handle.getState() != HandleState.CLOSED) {
-                                if (!handle.full()) {
-                                    while (true) {
-                                        ByteBuffer buffer = handle.getOutBuffer();
-                                        buffer.compact();
-                                        boolean success = msg.write(buffer);
-                                        buffer.flip();
-                                        if (success) {
-                                            system.getPerformanceCounters().addMessageSentCount(1);
-                                            SelectionKey key = handle.getKey();
-                                            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
-                                        } else {
-                                            if (!buffer.hasRemaining()) {
-                                                handle.resizeOutBuffer();
-                                                continue;
-                                            }
-                                            handle.markFull();
-                                            unsentMessagesBitmap.set(i);
-                                        }
-                                        break;
-                                    }
-                                } else {
-                                    unsentMessagesBitmap.set(i);
-                                }
-                            }
-                        }
-                        copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+                        sendPendingMessages();
                     }
                     if (n > 0) {
-                        for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
-                            SelectionKey key = i.next();
-                            i.remove();
-                            final SelectableChannel sc = key.channel();
-                            if (key.isReadable()) {
-                                read(key);
-                            } else if (key.isWritable()) {
-                                write(key);
-                            } else if (key.isAcceptable()) {
-                                assert sc == serverSocketChannel;
-                                SocketChannel channel = serverSocketChannel.accept();
-                                register(channel);
-                                IPCHandle handle = new IPCHandle(system, null);
-                                SelectionKey cKey = channel.register(selector, SelectionKey.OP_READ);
-                                handle.setKey(cKey);
-                                cKey.attach(handle);
-                                handle.setState(HandleState.CONNECT_RECEIVED);
-                            } else if (key.isConnectable()) {
-                                SocketChannel channel = (SocketChannel) sc;
-                                IPCHandle handle = (IPCHandle) key.attachment();
-                                if (!finishConnect(channel)) {
-                                    handle.setState(HandleState.CONNECT_FAILED);
-                                    continue;
-                                }
-
-                                handle.setState(HandleState.CONNECT_SENT);
-                                registerHandle(handle);
-                                key.interestOps(SelectionKey.OP_READ);
-                                IPCConnectionManager.this.write(createInitialReqMessage(handle));
-                            }
-                        }
+                        processSelectedKeys();
                     }
                     // reset failingLoops on a good loop
                     failingLoops = 0;
@@ -314,25 +215,146 @@
             }
         }
 
-        private void cleanup() {
-            for (Channel channel : openChannels) {
-                IOUtils.closeQuietly(channel);
+        private void processSelectedKeys() {
+            for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
+                SelectionKey key = i.next();
+                i.remove();
+                final SelectableChannel sc = key.channel();
+                if (key.isReadable()) {
+                    read(key);
+                } else if (key.isWritable()) {
+                    write(key);
+                } else if (key.isAcceptable()) {
+                    assert sc == serverSocketChannel;
+                    accept();
+                } else if (key.isConnectable()) {
+                    finishConnect(key);
+                }
             }
-            openChannels.clear();
-            IOUtils.closeQuietly(selector);
         }
 
-        private boolean finishConnect(SocketChannel channel) {
-            boolean connectFinished = false;
+        private void finishConnect(SelectionKey connectableKey) {
+            SocketChannel channel = (SocketChannel) connectableKey.channel();
+            IPCHandle handle = (IPCHandle) connectableKey.attachment();
+            boolean connected = false;
             try {
-                connectFinished = channel.finishConnect();
-                if (!connectFinished) {
-                    LOGGER.log(Level.WARN, "Channel connect did not finish");
+                connected = channel.finishConnect();
+                if (connected) {
+                    connectableKey.interestOps(SelectionKey.OP_READ);
+                    connectionEstablished(handle);
                 }
             } catch (IOException e) {
-                LOGGER.log(Level.WARN, "Exception finishing channel connect", e);
+                LOGGER.warn("Exception finishing connect", e);
+            } finally {
+                if (!connected) {
+                    LOGGER.warn("Failed to finish connect to {}", handle.getRemoteAddress());
+                    close(connectableKey, channel);
+                }
             }
-            return connectFinished;
+        }
+
+        private void accept() {
+            SocketChannel channel = null;
+            SelectionKey channelKey = null;
+            try {
+                channel = serverSocketChannel.accept();
+                register(channel);
+                channelKey = channel.register(selector, SelectionKey.OP_READ);
+                IPCHandle handle = new IPCHandle(system, null);
+                handle.setKey(channelKey);
+                channelKey.attach(handle);
+                handle.setState(HandleState.CONNECT_RECEIVED);
+            } catch (IOException e) {
+                LOGGER.error("Failed to accept channel ", e);
+                close(channelKey, channel);
+            }
+        }
+
+        private void establishPendingConnections() {
+            for (IPCHandle handle : workingPendingConnections) {
+                SocketChannel channel = null;
+                SelectionKey channelKey = null;
+                try {
+                    channel = SocketChannel.open();
+                    register(channel);
+                    if (channel.connect(handle.getRemoteAddress())) {
+                        channelKey = channel.register(selector, SelectionKey.OP_READ);
+                        connectionEstablished(handle);
+                    } else {
+                        channelKey = channel.register(selector, SelectionKey.OP_CONNECT);
+                    }
+                    handle.setKey(channelKey);
+                    channelKey.attach(handle);
+                } catch (IOException e) {
+                    LOGGER.error("Failed to accept channel ", e);
+                    close(channelKey, channel);
+                }
+            }
+            workingPendingConnections.clear();
+        }
+
+        private void connectionEstablished(IPCHandle handle) {
+            handle.setState(HandleState.CONNECT_SENT);
+            registerHandle(handle);
+            IPCConnectionManager.this.write(createInitialReqMessage(handle));
+        }
+
+        private void sendPendingMessages() {
+            unsentMessagesBitmap.clear();
+            int len = workingSendList.size();
+            for (int i = 0; i < len; ++i) {
+                Message msg = workingSendList.get(i);
+                final boolean sent = sendMessage(msg);
+                if (!sent) {
+                    unsentMessagesBitmap.set(i);
+                }
+            }
+            copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+        }
+
+        private boolean sendMessage(Message msg) {
+            LOGGER.debug("Processing send of message: {}", msg);
+            IPCHandle handle = msg.getIPCHandle();
+            if (handle.getState() == HandleState.CLOSED) {
+                // message will never be sent
+                return true;
+            }
+            if (handle.full()) {
+                return false;
+            }
+            try {
+                while (true) {
+                    ByteBuffer buffer = handle.getOutBuffer();
+                    buffer.compact();
+                    boolean success = msg.write(buffer);
+                    buffer.flip();
+                    if (success) {
+                        system.getPerformanceCounters().addMessageSentCount(1);
+                        SelectionKey key = handle.getKey();
+                        key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                        return true;
+                    } else {
+                        if (!buffer.hasRemaining()) {
+                            handle.resizeOutBuffer();
+                            continue;
+                        }
+                        handle.markFull();
+                        return false;
+                    }
+                }
+            } catch (Exception e) {
+                LOGGER.fatal("Unrecoverable networking failure; Halting...", e);
+                ExitUtil.halt(EC_IMMEDIATE_HALT);
+            }
+            return false;
+        }
+
+        private void cleanup() {
+            for (Channel channel : openChannels) {
+                NetworkUtil.closeQuietly(channel);
+            }
+            openChannels.clear();
+            NetworkUtil.closeQuietly(selector);
         }
 
         private void copyUnsentMessages(BitSet unsentMessagesBitmap, List<Message> tempUnsentMessages) {
@@ -396,19 +418,42 @@
         }
 
         private void close(SelectionKey key, SocketChannel sc) {
-            key.cancel();
-            NetworkUtil.closeQuietly(sc);
-            openChannels.remove(sc);
-            final IPCHandle handle = (IPCHandle) key.attachment();
-            handle.close();
+            if (key != null) {
+                final Object attachment = key.attachment();
+                if (attachment != null) {
+                    ((IPCHandle) attachment).close();
+                }
+                key.cancel();
+            }
+            if (sc != null) {
+                NetworkUtil.closeQuietly(sc);
+                openChannels.remove(sc);
+            }
         }
-    }
 
-    private <T> void moveAll(List<T> source, List<T> target) {
-        int len = source.size();
-        for (int i = 0; i < len; ++i) {
-            target.add(source.get(i));
+        private void collectOutstandingWork() {
+            synchronized (IPCConnectionManager.this) {
+                if (!pendingConnections.isEmpty()) {
+                    moveAll(pendingConnections, workingPendingConnections);
+                }
+                if (!sendList.isEmpty()) {
+                    moveAll(sendList, workingSendList);
+                }
+            }
         }
-        source.clear();
+
+        private Message createInitialReqMessage(IPCHandle handle) {
+            Message msg = new Message(handle);
+            msg.setMessageId(system.createMessageId());
+            msg.setRequestMessageId(-1);
+            msg.setFlag(Message.INITIAL_REQ);
+            msg.setPayload(address);
+            return msg;
+        }
+
+        private <T> void moveAll(List<T> source, List<T> target) {
+            target.addAll(source);
+            source.clear();
+        }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
index d63bfbd..09c7c97 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
@@ -62,10 +62,6 @@
         return system;
     }
 
-    void setRemoteAddress(InetSocketAddress remoteAddress) {
-        this.remoteAddress = remoteAddress;
-    }
-
     @Override
     public long send(long requestId, Object req, Exception exception) throws IPCException {
         if (!isConnected()) {
@@ -127,7 +123,6 @@
                     wait();
                     break;
                 case CONNECTED:
-                case CONNECT_FAILED:
                 case CLOSED:
                     return state == HandleState.CONNECTED;
                 default:
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
index b36e645..b7dcf05 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
@@ -66,10 +66,6 @@
         cMgr.stop();
     }
 
-    public IIPCHandle getHandle(InetSocketAddress remoteAddress) throws IPCException {
-        return getHandle(remoteAddress, 0);
-    }
-
     public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries) throws IPCException {
         return getHandle(remoteAddress, maxRetries, 0);
     }
@@ -78,16 +74,6 @@
         return getReconnectingHandle(remoteAddress, 1);
     }
 
-    public IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
-            throws IPCException {
-        return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
-    }
-
-    public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
-            throws IPCException {
-        return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
-    }
-
     public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts,
             IIPCEventListener eventListener) throws IPCException {
         if (reconnectAttempts > 0) {
@@ -132,4 +118,14 @@
     public IPCPerformanceCounters getPerformanceCounters() {
         return perfCounters;
     }
+
+    private IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
+            throws IPCException {
+        return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+    }
+
+    private IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
+            throws IPCException {
+        return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
index cca3abe..70a0e18 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
@@ -43,7 +43,7 @@
         IPCSystem client = createClientIPCSystem(rpci);
         client.start();
 
-        IIPCHandle handle = client.getHandle(serverAddr);
+        IIPCHandle handle = client.getHandle(serverAddr, 0);
 
         for (int i = 0; i < 100; ++i) {
             Assert.assertEquals(rpci.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
index 475ed50..17eba93 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
@@ -147,7 +147,12 @@
             getLog().debug("Resolving content for " + artifact.getUrl() + " (" + artifact.getContentFile() + ")");
             File cFile = new File(artifact.getContentFile());
             if (!cFile.isAbsolute()) {
-                cFile = new File(licenseDirectory, artifact.getContentFile());
+                for (File directory : licenseDirectories) {
+                    cFile = new File(directory, artifact.getContentFile());
+                    if (cFile.exists()) {
+                        break;
+                    }
+                }
             }
             if (!cFile.exists()) {
                 if (!bestEffort) {
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
index a94727d..05ac62b 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
@@ -74,6 +74,9 @@
     protected List<LicenseSpec> licenses = new ArrayList<>();
 
     @Parameter
+    protected List<NoticeSpec> notices = new ArrayList<>();
+
+    @Parameter
     protected Set<String> excludedScopes = new HashSet<>();
 
     @Parameter
@@ -106,7 +109,10 @@
     @Parameter(required = true)
     private String location;
 
-    @Parameter(required = true)
+    @Parameter
+    protected List<File> licenseDirectories = new ArrayList<>();
+
+    @Parameter
     protected File licenseDirectory;
 
     @Parameter
@@ -122,6 +128,7 @@
     private List<Pattern> excludePatterns;
 
     Map<String, LicenseSpec> urlToLicenseMap = new HashMap<>();
+    Map<String, NoticeSpec> urlToNoticeMap = new HashMap<>();
     Map<String, LicensedProjects> licenseMap = new TreeMap<>();
     private Map<Pair<String, ProjectFlag>, Object> projectFlags = new HashMap<>();
     Map<String, String> noticeOverrides = new HashMap<String, String>();
@@ -133,6 +140,9 @@
     }
 
     protected void init() throws MojoExecutionException {
+        if (licenseDirectory != null) {
+            licenseDirectories.add(0, licenseDirectory);
+        }
         if (warningTouchFile != null) {
             warningTouchFile.getParentFile().mkdirs();
         }
@@ -140,7 +150,7 @@
         excludedScopes.add("system");
         excludePatterns = compileExcludePatterns();
         supplementModels = SupplementalModelHelper.loadSupplements(getLog(), models);
-        buildUrlLicenseMap();
+        buildUrlMaps();
     }
 
     private void interceptLogs() {
@@ -321,7 +331,7 @@
         }
     }
 
-    private void buildUrlLicenseMap() throws MojoExecutionException {
+    private void buildUrlMaps() throws MojoExecutionException {
         for (LicenseSpec license : licenses) {
             if (urlToLicenseMap.put(license.getUrl(), license) != null) {
                 throw new MojoExecutionException("Duplicate URL mapping: " + license.getUrl());
@@ -332,6 +342,16 @@
                 }
             }
         }
+        for (NoticeSpec notice : notices) {
+            if (urlToNoticeMap.put(notice.getUrl(), notice) != null) {
+                throw new MojoExecutionException("Duplicate URL mapping: " + notice.getUrl());
+            }
+            for (String alias : notice.getAliasUrls()) {
+                if (urlToNoticeMap.put(alias, notice) != null) {
+                    throw new MojoExecutionException("Duplicate URL mapping: " + alias);
+                }
+            }
+        }
     }
 
     protected Map<MavenProject, List<Pair<String, String>>> gatherDependencies() throws ProjectBuildingException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 49eca6f..566d8e2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 
 public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -48,6 +49,8 @@
     protected boolean appendOpCallbackProceedResult;
     protected byte[] searchCallbackProceedResultFalseValue;
     protected byte[] searchCallbackProceedResultTrueValue;
+    protected final ITupleFilterFactory tupleFilterFactory;
+    protected final long outputLimit;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -56,7 +59,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
         this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
     }
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -64,8 +67,8 @@
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
-            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -80,6 +83,8 @@
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
         this.outRecDescs[0] = outRecDesc;
+        this.tupleFilterFactory = tupleFilterFactory;
+        this.outputLimit = outputLimit;
         this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
         this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
         this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
@@ -92,7 +97,7 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 6bbf437..70b88fe 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -50,7 +51,7 @@
             throws HyracksDataException {
         this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
-                missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null, null);
+                missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
     }
 
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -58,11 +59,12 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
index 50673cf..b5dfe00 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
@@ -19,8 +19,9 @@
 
 package org.apache.hyracks.storage.am.common.api;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public interface ITupleFilter {
-    public boolean accept(IFrameTupleReference tuple) throws Exception;
+    public boolean accept(IFrameTupleReference tuple) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
index 16c0afa..c7be7eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
@@ -39,6 +39,6 @@
     /**
      * Abort the bulk modify op
      */
-    public void abort();
+    public void abort() throws HyracksDataException;
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index 2c691f1..fb287cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IMissingWriter;
@@ -37,11 +38,14 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilter;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -95,22 +99,43 @@
     protected byte[] searchCallbackProceedResultFalseValue;
     protected byte[] searchCallbackProceedResultTrueValue;
 
+    protected final ITupleFilterFactory tupleFilterFactory;
+    protected ReferenceFrameTupleReference referenceFilterTuple;
+    // filter out tuples based on the query-provided select condition
+    // only results satisfying the filter condition would be returned to downstream operators
+    protected ITupleFilter tupleFilter;
+    protected final long outputLimit;
+    protected long outputCount = 0;
+    protected boolean finished;
+
+    // no filter and limit pushdown
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
             throws HyracksDataException {
         this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
-                null);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
+                false, null, null);
     }
 
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+        this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                tupleFilterFactory, outputLimit, false, null, null);
+    }
+
+    public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+            throws HyracksDataException {
         this.ctx = ctx;
         this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
         this.retainInput = retainInput;
@@ -138,6 +163,12 @@
         if (ctx.getStatsCollector() != null) {
             ctx.getStatsCollector().add(stats);
         }
+        this.tupleFilterFactory = tupleFactoryFactory;
+        this.outputLimit = outputLimit;
+
+        if (this.tupleFilterFactory != null && this.retainMissing) {
+            throw new IllegalStateException("RetainMissing with tuple filter is not supported");
+        }
     }
 
     protected abstract ISearchPredicate createSearchPredicate();
@@ -179,6 +210,13 @@
             buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
         }
 
+        if (tupleFilterFactory != null) {
+            tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
+            referenceFilterTuple = new ReferenceFrameTupleReference();
+        }
+        finished = false;
+        outputCount = 0;
+
         try {
             searchPred = createSearchPredicate();
             tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
@@ -201,9 +239,14 @@
     protected void writeSearchResults(int tupleIndex) throws Exception {
         long matchingTupleCount = 0;
         while (cursor.hasNext()) {
-            matchingTupleCount++;
-            tb.reset();
             cursor.next();
+            matchingTupleCount++;
+            ITupleReference tuple = cursor.getTuple();
+            if (tupleFilter != null && !tupleFilter.accept(referenceFilterTuple.reset(tuple))) {
+                continue;
+            }
+            tb.reset();
+
             if (retainInput) {
                 frameTuple.reset(accessor, tupleIndex);
                 for (int i = 0; i < frameTuple.getFieldCount(); i++) {
@@ -211,7 +254,6 @@
                     tb.addFieldEndOffset();
                 }
             }
-            ITupleReference tuple = cursor.getTuple();
             writeTupleToOutput(tuple);
             if (appendSearchCallbackProceedResult) {
                 writeSearchCallbackProceedResult(tb,
@@ -222,6 +264,10 @@
                 writeFilterTupleToOutput(((ILSMIndexCursor) cursor).getFilterMaxTuple());
             }
             FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+            if (outputLimit >= 0 && ++outputCount >= outputLimit) {
+                finished = true;
+                break;
+            }
         }
         stats.getTupleCounter().update(matchingTupleCount);
 
@@ -237,7 +283,7 @@
         accessor.reset(buffer);
         int tupleCount = accessor.getTupleCount();
         try {
-            for (int i = 0; i < tupleCount; i++) {
+            for (int i = 0; i < tupleCount && !finished; i++) {
                 resetSearchPredicate(i);
                 cursor.close();
                 indexAccessor.search(cursor, searchPred);
@@ -344,4 +390,50 @@
         }
     }
 
+    /**
+     * A wrapper class to wrap ITupleReference into IFrameTupleReference, as the latter
+     * is used by ITupleFilter
+     *
+     */
+    private static class ReferenceFrameTupleReference implements IFrameTupleReference {
+        private ITupleReference tuple;
+
+        public IFrameTupleReference reset(ITupleReference tuple) {
+            this.tuple = tuple;
+            return this;
+        }
+
+        @Override
+        public int getFieldCount() {
+            return tuple.getFieldCount();
+        }
+
+        @Override
+        public byte[] getFieldData(int fIdx) {
+            return tuple.getFieldData(fIdx);
+        }
+
+        @Override
+        public int getFieldStart(int fIdx) {
+            return tuple.getFieldStart(fIdx);
+        }
+
+        @Override
+        public int getFieldLength(int fIdx) {
+            return tuple.getFieldLength(fIdx);
+        }
+
+        @Override
+        public IFrameTupleAccessor getFrameTupleAccessor() {
+            throw new UnsupportedOperationException(
+                    "getFrameTupleAccessor is not supported by ReferenceFrameTupleReference");
+        }
+
+        @Override
+        public int getTupleIndex() {
+            throw new UnsupportedOperationException("getTupleIndex is not supported by ReferenceFrameTupleReference");
+        }
+
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
index 636e4f5..56600b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -35,6 +35,5 @@
     FLUSH,
     REPLICATE,
     DISK_COMPONENT_SCAN,
-    DELETE_MEMORY_COMPONENT,
-    DELETE_DISK_COMPONENTS
+    DELETE_COMPONENTS
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
index 18de540..c635411 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
@@ -19,8 +19,6 @@
 
 package org.apache.hyracks.storage.am.common.tuples;
 
-import java.util.Arrays;
-
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 /**
@@ -96,14 +94,17 @@
         return tuples[tupleIndex].getFieldLength(fieldIndex);
     }
 
+    /**
+     * Right now this class is only used by inverted index, and only contains 2 tuples.
+     * As a result, sequential search would be more efficient than binary search
+     */
     private int getTupleIndex(int fIdx) {
-        int tupleIndex = Arrays.binarySearch(fieldCounts, 0, numTuples - 1, fIdx);
-        if (tupleIndex < 0) {
-            tupleIndex = -tupleIndex - 1;
-        } else {
-            ++tupleIndex;
+        for (int i = 0; i < numTuples; i++) {
+            if (fIdx < fieldCounts[i]) {
+                return i;
+            }
         }
-        return tupleIndex;
+        throw new IllegalArgumentException("Illegal field index " + fIdx);
     }
 
     private int getFieldIndex(int tupleIndex, int fIdx) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
index c0f7571..a504f7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -22,6 +22,7 @@
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -40,8 +41,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -54,6 +53,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -170,8 +170,7 @@
     // The only reason to override the following method is that it uses a different context object
     // in addition, determining whether or not to keep deleted tuples is different here
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public LSMBTreeMergeOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         opCtx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -195,9 +194,11 @@
         LSMComponentFileReferences relMergeFileRefs =
                 fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-        ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, cursor,
+        LSMBTreeMergeOperation mergeOp = new LSMBTreeMergeOperation(accessor, cursor,
                 relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath()));
+                ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     // This function should only be used when a transaction fail. it doesn't
@@ -286,11 +287,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -353,8 +349,7 @@
 
     // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
     }
 
@@ -401,16 +396,16 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     // The bulk loader used for both initial loading and transaction
@@ -418,21 +413,37 @@
     public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
         private final ILSMDiskComponentBulkLoader componentBulkLoader;
+        private final LoadOperation loadOp;
 
         private final boolean isTransaction;
 
         public LSMTwoPCBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component = createDiskComponent(transactionComponentFactory,
+                        componentFileRefs.getInsertIndexFileReference(), null,
+                        componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
-
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
             componentBulkLoader =
-                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, true);
         }
 
         // It is expected that the mode was set to insert operation before
@@ -444,16 +455,23 @@
 
         @Override
         public void end() throws HyracksDataException {
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        component.markAsValid(durable);
+                        ioOpCallback.afterFinalize(loadOp);
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(component);
+                    }
                 }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
@@ -465,26 +483,13 @@
         }
 
         @Override
-        public void abort() {
+        public void abort() throws HyracksDataException {
             try {
                 componentBulkLoader.abort();
-            } catch (Exception e) {
-                // Do nothing
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
-
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    null, componentFileRefs.getBloomFilterFileReference(), true);
-        }
     }
 
     // The accessor for disk only indexes don't use modification callback and always carry the target index version with them
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index 1ba55f7..b727a39 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -35,7 +36,6 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
@@ -44,7 +44,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -59,6 +58,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -217,16 +217,16 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     @Override
@@ -244,8 +244,7 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
     }
 
@@ -267,7 +266,7 @@
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+    public LSMBTreeWithBuddyMergeOperation createMergeOperation(ILSMIndexOperationContext ctx)
             throws HyracksDataException {
         ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
         bctx.setOperation(IndexOperation.MERGE);
@@ -289,10 +288,13 @@
                     .get(secondDiskComponents.size() - 1);
         }
 
-        ioScheduler.scheduleOperation(
+        LSMBTreeWithBuddyMergeOperation mergeOp =
                 new LSMBTreeWithBuddyMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
                         relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                        callback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples));
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples);
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+
     }
 
     // This method creates the appropriate opContext for the targeted version
@@ -331,7 +333,8 @@
                 numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
                         .getBloomFilter().getNumElements();
             }
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+            componentBulkLoader =
+                    mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             try {
                 while (buddyBtreeCursor.hasNext()) {
                     buddyBtreeCursor.next();
@@ -342,7 +345,7 @@
                 buddyBtreeCursor.close();
             }
         } else {
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
 
         try {
@@ -430,11 +433,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         // Even though, we deactivate the index, we don't exit components or
         // modify any of the lists to make sure they
         // are there if the index was opened again
@@ -497,21 +495,39 @@
     // modifications
     public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
+        private final LoadOperation loadOp;
         private final ILSMDiskComponentBulkLoader componentBulkLoader;
         private final boolean isTransaction;
 
         public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
 
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
             componentBulkLoader =
-                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, false);
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
@@ -521,16 +537,23 @@
 
         @Override
         public void end() throws HyracksDataException {
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        component.markAsValid(durable);
+                        ioOpCallback.afterFinalize(loadOp);
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(component);
+                    }
                 }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
@@ -540,26 +563,17 @@
         }
 
         @Override
-        public void abort() {
+        public void abort() throws HyracksDataException {
             try {
-                componentBulkLoader.abort();
-            } catch (Exception e) {
+                try {
+                    componentBulkLoader.abort();
+                } finally {
+                    ioOpCallback.afterFinalize(loadOp);
+                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
-
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then eventually be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                    true);
-        }
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 41a11e6..5332b1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -74,7 +74,7 @@
 
 public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
-    private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
+    private static final ICursorFactory cursorFactory = LSMBTreeSearchCursor::new;
     // Common for in-memory and on-disk components.
     protected final ITreeIndexFrameFactory insertLeafFrameFactory;
     protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
@@ -109,8 +109,7 @@
                     new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), interiorFrameFactory,
                             insertLeafFrameFactory, cmpFactories, fieldCount,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_" + i)),
-                    virtualBufferCache, i == 0 ? true : false,
-                    filterHelper == null ? null : filterHelper.createFilter());
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -277,7 +276,7 @@
             }
             component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
                     true);
-            componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             IIndexCursor scanCursor = accessor.createSearchCursor(false);
             accessor.search(scanCursor, nullPred);
             try {
@@ -337,7 +336,7 @@
                     mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
                     componentBulkLoader =
-                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                            mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
                     while (cursor.hasNext()) {
                         cursor.next();
                         ITupleReference frameTuple = cursor.getTuple();
@@ -391,7 +390,7 @@
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback) {
         ILSMIndexAccessor accessor = createAccessor(opCtx);
         return new LSMBTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
@@ -476,6 +475,6 @@
         }
         LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
         return new LSMBTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
-                mergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
index 7acc59f..2ad4fee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -26,11 +26,13 @@
 
 public class LSMBTreeFlushOperation extends FlushOperation {
     private final FileReference bloomFilterFlushTarget;
+    private final LSMComponentFileReferences fileReferences;
 
     public LSMBTreeFlushOperation(ILSMIndexAccessor accessor, FileReference flushTarget,
             FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
         super(accessor, flushTarget, callback, indexIdentifier);
         this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+        fileReferences = new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
     }
 
     public FileReference getBloomFilterTarget() {
@@ -39,6 +41,6 @@
 
     @Override
     public LSMComponentFileReferences getComponentFiles() {
-        return new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
+        return fileReferences;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
index c7d555d..831562c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
@@ -28,9 +28,9 @@
 
     private final BTree btree;
 
-    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc, boolean isActive,
+    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc,
             ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1cfc414..1312e30 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -32,7 +32,6 @@
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -40,7 +39,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 361612e..fb0b9ac0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -105,49 +105,55 @@
         while (!outputPriorityQueue.isEmpty() || needPushElementIntoQueue) {
             if (!outputPriorityQueue.isEmpty()) {
                 PriorityQueueElement queueHead = outputPriorityQueue.peek();
-                if (canCallProceed && includeMutableComponent) {
-                    resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
-                    if (!resultOfSearchCallbackProceed) {
-                        // In case proceed() fails and there is an in-memory component,
-                        // we can't simply use this element since there might be a change.
-                        PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
-                        if (mutableElement != null) {
-                            // Copies the current queue head
-                            if (tupleBuilder == null) {
-                                tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                if (canCallProceed) {
+                    if (includeMutableComponent) {
+                        resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
+                        if (!resultOfSearchCallbackProceed) {
+                            // In case proceed() fails and there is an in-memory component,
+                            // we can't simply use this element since there might be a change.
+                            PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
+                            if (mutableElement != null) {
+                                // Copies the current queue head
+                                if (tupleBuilder == null) {
+                                    tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                                }
+                                TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
+                                copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+                                // Unlatches/unpins the leaf page of the index.
+                                rangeCursors[0].close();
+                                // Reconcile.
+                                searchCallback.reconcile(copyTuple);
+                                // Re-traverses the index.
+                                reusablePred.setLowKey(copyTuple, true);
+                                btreeAccessors[0].search(rangeCursors[0], reusablePred);
+                                pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
+                                // now that we have completed the search and we have latches over the pages,
+                                // it is safe to complete the operation.. but as per the API of the callback
+                                // we only complete if we're producing this tuple
+                                // get head again
+                                queueHead = outputPriorityQueue.peek();
+                                /*
+                                 * We need to restart in one of two cases:
+                                 * 1. no more elements in the priority queue.
+                                 * 2. the key of the head has changed (which means we need to call proceed)
+                                 */
+                                if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
+                                    // cancel since we're not continuing
+                                    searchCallback.cancel(copyTuple);
+                                    continue;
+                                }
+                                searchCallback.complete(copyTuple);
+                                // it is safe to proceed now
+                            } else {
+                                // There are no more elements in the memory component.. can safely skip locking for the
+                                // remaining operations
+                                includeMutableComponent = false;
                             }
-                            TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
-                            copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-                            // Unlatches/unpins the leaf page of the index.
-                            rangeCursors[0].close();
-                            // Reconcile.
-                            searchCallback.reconcile(copyTuple);
-                            // Re-traverses the index.
-                            reusablePred.setLowKey(copyTuple, true);
-                            btreeAccessors[0].search(rangeCursors[0], reusablePred);
-                            pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
-                            // now that we have completed the search and we have latches over the pages,
-                            // it is safe to complete the operation.. but as per the API of the callback
-                            // we only complete if we're producing this tuple
-                            // get head again
-                            queueHead = outputPriorityQueue.peek();
-                            /*
-                             * We need to restart in one of two cases:
-                             * 1. no more elements in the priority queue.
-                             * 2. the key of the head has changed (which means we need to call proceed)
-                             */
-                            if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
-                                // cancel since we're not continuing
-                                searchCallback.cancel(copyTuple);
-                                continue;
-                            }
-                            searchCallback.complete(copyTuple);
-                            // it is safe to proceed now
-                        } else {
-                            // There are no more elements in the memory component.. can safely skip locking for the
-                            // remaining operations
-                            includeMutableComponent = false;
                         }
+                    } else {
+                        // only perform locking for tuples from memory components.
+                        // all tuples from disk components have already been committed, and we're safe to proceed
+                        resultOfSearchCallbackProceed = true;
                     }
                 }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index baf0d4a7..efacad1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -107,6 +107,6 @@
 
     @Override
     public boolean getSearchOperationCallbackProceedResult() {
-        return false;
+        return currentCursor.getSearchOperationCallbackProceedResult();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
index a1a1e45..ed2ee70 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
@@ -20,10 +20,10 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 
 /*
  * This class is also not needed at the moment but is implemented anyway
@@ -34,8 +34,8 @@
     private final BTree buddyBtree;
 
     public LSMBTreeWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, BTree btree, BTree buddyBtree,
-            IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
         this.buddyBtree = buddyBtree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index cc10a98..d071bac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -153,29 +153,24 @@
         // can be used for both inserts and deletes
         ITreeIndexFrameFactory transactionLeafFrameFactory =
                 new BTreeNSMLeafFrameFactory(transactionTupleWriterFactory);
-
         TreeIndexFactory<DiskBTree> diskBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
         TreeIndexFactory<DiskBTree> bulkLoadBTreeFactory = new DiskBTreeFactory(ioManager, diskBufferCache,
                 freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-
         BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-
         // This is the component factory for transactions
         TreeIndexFactory<DiskBTree> transactionBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         transactionLeafFrameFactory, cmpFactories, typeTraits.length);
         //TODO remove BloomFilter from external dataset's secondary LSMBTree index
         ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, true);
-
         ILSMDiskComponentFactory componentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory bulkLoadComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory transactionComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
-
         // the disk only index uses an empty ArrayList for virtual buffer caches
         return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
                 diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
index 107190d..329a54b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -59,13 +59,6 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        super.deactivateAndDestroy();
-        getBloomFilter().deactivate();
-        getBloomFilter().destroy();
-    }
-
-    @Override
     public void destroy() throws HyracksDataException {
         super.destroy();
         getBloomFilter().destroy();
@@ -78,9 +71,8 @@
     }
 
     @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        super.deactivateAndPurge();
-        getBloomFilter().deactivate();
+    protected void purge() throws HyracksDataException {
+        super.purge();
         getBloomFilter().purge();
     }
 
@@ -92,10 +84,10 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
-        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(fillFactor, verifyInput,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
                 numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
         if (numElementsHint > 0) {
             chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
index c2f52e0..cace9e5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
@@ -52,13 +52,6 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        super.deactivateAndDestroy();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-    }
-
-    @Override
     public void destroy() throws HyracksDataException {
         super.destroy();
         getBuddyIndex().destroy();
@@ -71,9 +64,8 @@
     }
 
     @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        super.deactivateAndPurge();
-        getBuddyIndex().deactivate();
+    protected void purge() throws HyracksDataException {
+        super.purge();
         getBuddyIndex().purge();
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
deleted file mode 100644
index 551303f..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.api;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-
-public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
-
-    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
-    }
-
-    public abstract AbstractTreeIndex getBuddyIndex();
-
-    @Override
-    public void doReset() throws HyracksDataException {
-        super.doReset();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-        getBuddyIndex().create();
-        getBuddyIndex().activate();
-    }
-
-    @Override
-    public void doAllocate() throws HyracksDataException {
-        super.doAllocate();
-        getBuddyIndex().create();
-        getBuddyIndex().activate();
-    }
-
-    @Override
-    public void doDeallocate() throws HyracksDataException {
-        super.doDeallocate();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-    }
-
-    @Override
-    public void validate() throws HyracksDataException {
-        super.validate();
-        getBuddyIndex().validate();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
new file mode 100644
index 0000000..a9b9cc0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+/**
+ * A callback that will be called on failures of IO operations
+ */
+public interface IIoOperationFailedCallback {
+    /**
+     * Called on a failed IO operation
+     *
+     * @param operation
+     *            the failed operation
+     * @param failure
+     *            the failure
+     */
+    void operationFailed(ILSMIOOperation operation, Throwable failure);
+
+    /**
+     * Called on a scheduler failure.
+     *
+     * @param scheduler
+     *            the failed scheduler
+     * @param failure
+     *            the failure
+     */
+    void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index fc55ce5..fef59e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.common.IIndex;
 
 /**
@@ -139,4 +140,17 @@
      * @throws HyracksDataException
      */
     ILSMComponentId getId() throws HyracksDataException;
+
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException;
+
+    /**
+     * @return the number of readers inside a component
+     */
+    int getReaderCount();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
index 5662862..c3835eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
@@ -38,4 +38,14 @@
     boolean missing();
 
     IdCompareResult compareTo(ILSMComponentId id);
+
+    /**
+     * @return the min Id
+     */
+    long getMinId();
+
+    /**
+     * @return the max Id
+     */
+    long getMaxId();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
index 5dd3061..e6aa2d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
@@ -26,12 +26,17 @@
     /**
      * @return An Id for LSM component
      */
-    public ILSMComponentId getId();
+    ILSMComponentId getId();
 
     /**
      * Refresh the component Id generator to generate the next Id.
      * {@link #getId()} would always return the same Id before this method is called.
      */
-    public void refresh();
+    void refresh();
+
+    /**
+     * @return the index of the current memory component
+     */
+    int getCurrentComponentIndex();
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
deleted file mode 100644
index 4ec82c1..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.api;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.common.IResource;
-
-@FunctionalInterface
-public interface ILSMComponentIdGeneratorFactory extends Serializable {
-    ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
-            throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
index bd2bb45..1500f37 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
@@ -50,6 +50,7 @@
     /**
      * @return LsmIndex of the component
      */
+    @Override
     AbstractLSMIndex getLsmIndex();
 
     /**
@@ -142,6 +143,7 @@
      * Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
      * component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
      *
+     * @param operation
      * @param fillFactor
      * @param verifyInput
      * @param numElementsHint
@@ -151,6 +153,7 @@
      * @return
      * @throws HyracksDataException
      */
-    ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent) throws HyracksDataException;
+    ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+            throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
index 13ce971..8284cee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
@@ -41,4 +41,9 @@
      */
     void cleanupArtifacts() throws HyracksDataException;
 
+    /**
+     * @return The operation this bulkLoader is operating under
+     */
+    ILSMIOOperation getOperation();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index b32dd0f..c4a0352 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -108,7 +108,7 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Schedule full merge
@@ -118,17 +118,16 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a merge operation
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void merge(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Schedule a flush
@@ -137,17 +136,16 @@
      * @param callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a flush
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void flush(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Add bulk loaded component
@@ -171,13 +169,11 @@
      *            the operation context
      * @param diskComponents
      *            the disk component to be replicated
-     * @param bulkload
-     *            true if the components were bulk loaded, false otherwise
      * @param opType
      *            The operation type
      * @throws HyracksDataException
      */
-    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
             LSMOperationType opType) throws HyracksDataException;
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 65e7f64..3245455 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -18,14 +18,16 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.api;
 
+import java.util.Map;
 import java.util.concurrent.Callable;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 
-public interface ILSMIOOperation extends Callable<Boolean> {
+public interface ILSMIOOperation extends Callable<LSMIOOperationStatus> {
 
     /**
      * Represents the io operation type
@@ -38,6 +40,20 @@
     }
 
     /**
+     * Represents the status of the IO operation
+     */
+    enum LSMIOOperationStatus {
+        /**
+         * Operation successful
+         */
+        SUCCESS,
+        /**
+         * Operation failed
+         */
+        FAILURE
+    }
+
+    /**
      * @return the device on which the operation is running
      */
     IODeviceHandle getDevice();
@@ -58,7 +74,7 @@
     LSMIOOperationType getIOOpertionType();
 
     @Override
-    Boolean call() throws HyracksDataException;
+    LSMIOOperationStatus call() throws HyracksDataException;
 
     /**
      * @return The target of the io operation
@@ -74,4 +90,62 @@
      * @return the component files produced by this operation
      */
     LSMComponentFileReferences getComponentFiles();
+
+    /**
+     * @return the failure in the io operation if any, null otherwise
+     */
+    Throwable getFailure();
+
+    /**
+     * @return set the failure in the io operation
+     */
+    void setFailure(Throwable failure);
+
+    /**
+     * @return the status of the IO operation
+     */
+    LSMIOOperationStatus getStatus();
+
+    /**
+     * Set the status of the IO operation
+     *
+     * @param status
+     */
+    void setStatus(LSMIOOperationStatus status);
+
+    /**
+     * @return the new component produced by this operation if any, null otherwise
+     */
+    ILSMDiskComponent getNewComponent();
+
+    /**
+     * Set the new component produced by this operation
+     *
+     * @param component
+     */
+    void setNewComponent(ILSMDiskComponent component);
+
+    /**
+     * Destroy the operation after the scheduler is done with it
+     */
+    void complete();
+
+    /**
+     * Wait for the operation to complete
+     *
+     * @throws InterruptedException
+     */
+    void sync() throws InterruptedException;
+
+    /**
+     * Add a listener for operation complete event
+     *
+     * @param listener
+     */
+    void addCompleteListener(IoOperationCompleteListener listener);
+
+    /**
+     * Get parameters passed when calling this IO operation
+     */
+    Map<String, Object> getParameters();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index acc9e89..e448ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -23,14 +23,23 @@
 public interface ILSMIOOperationCallback {
 
     /**
-     * This method is called on an IO operation before the operation starts.
-     * (i.e. IO operations could be flush, or merge operations.)
-     * For flush, this is called immediately before switching the current memory component pointer
+     * This method is called on an IO operation before the operation is scheduled
+     * For operations that are not scheduled(i,e. Bulk load), this call is skipped.
+     *
+     * @param operation
+     * @throws HyracksDataException
      */
-    void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void scheduled(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
-     * This method is called on an IO operation sometime after the operation was completed.
+     * This method is called on an IO operation before the operation starts.
+     * (i.e. IO operations could be flush, or merge operations.)
+     */
+    void beforeOperation(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called on an IO operation sometime after the operation is completed but before the new component
+     * is marked as valid.
      * (i.e. IO operations could be flush or merge operations.)
      *
      * Copying content of metadata page from memory component to disk component should be done in this call
@@ -38,31 +47,37 @@
      *
      * @throws HyracksDataException
      */
-    void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void afterOperation(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * This method is called on an IO operation when the operation needs any cleanup works
      * regardless that the IO operation was executed or not. Once the IO operation is executed,
      * this method should be called after ILSMIOOperationCallback.afterOperation() was called.
      *
+     */
+    void afterFinalize(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called after the schduler is done with the IO operation
+     * For operation that are not scheduled, this call is skipped
+     *
+     * @param operation
      * @throws HyracksDataException
      */
-    void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void completed(ILSMIOOperation operation);
 
     /**
      * This method is called when a memory component is recycled
      *
      * @param component
-     * @param componentSwitched
-     *            true if the component index was advanced for this recycle, false otherwise
      */
-    void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException;
+    void recycled(ILSMMemoryComponent component) throws HyracksDataException;
 
     /**
      * This method is called when a memory component is allocated
      *
      * @param component
+     *            the allocated component
      */
     void allocated(ILSMMemoryComponent component) throws HyracksDataException;
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index e8742b5..8ccf41f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -32,5 +32,18 @@
      */
     void initialize(INCServiceContext ncCtx, IResource resource);
 
+    /**
+     * Create the IO Operation Callback
+     *
+     * @param index
+     * @return
+     * @throws HyracksDataException
+     */
     ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException;
+
+    /**
+     * @return the current memory component index
+     * @throws HyracksDataException
+     */
+    int getCurrentMemoryComponentIndex() throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index a8467d3..5cb05a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
@@ -28,6 +29,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMHarness;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 
@@ -48,8 +50,6 @@
 
     ILSMOperationTracker getOperationTracker();
 
-    ILSMIOOperationScheduler getIOScheduler();
-
     ILSMIOOperationCallback getIOOperationCallback();
 
     /**
@@ -74,11 +74,33 @@
 
     public void scanDiskComponents(ILSMIndexOperationContext ctx, IIndexCursor cursor) throws HyracksDataException;
 
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a flush operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @return the flush operation
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException;
 
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a merge operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException;
 
@@ -108,7 +130,7 @@
 
     boolean isCurrentMutableComponentEmpty() throws HyracksDataException;
 
-    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
             ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException;
 
     boolean isMemoryComponentsAllocated();
@@ -157,4 +179,36 @@
      * @return the {@link ILSMHarness} of the index
      */
     ILSMHarness getHarness();
+
+    /**
+     * Cleanup the files of the failed operation
+     *
+     * @param operation
+     */
+    void cleanUpFilesForFailedOperation(ILSMIOOperation operation);
+
+    /**
+     * @return the absolute path of the index
+     */
+    String getIndexIdentifier();
+
+    /**
+     * Create a bulk loader
+     *
+     * @param fillFactor
+     * @param verifyInput
+     * @param numElementsHint
+     * @param checkIfEmptyIndex
+     * @param parameters
+     * @return
+     * @throws HyracksDataException
+     */
+    IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException;
+
+    /**
+     * Reset the current memory component id to 0.
+     */
+    void resetCurrentComponentIndex();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index 61ef6cf..42d3ab7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -44,34 +44,27 @@
     /**
      * Schedule a flush operation
      *
-     * @param callback
-     *            the IO operation callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush() throws HyracksDataException;
 
     /**
      * Schedule a merge operation
      *
-     * @param callback
-     *            the merge operation callback
      * @param components
      *            the components to be merged
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException;
 
     /**
      * Schedule a full merge
      *
-     * @param callback
-     *            the merge operation callback
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge() throws HyracksDataException;
 
     /**
      * Delete the tuple from the memory component only. Don't replace with antimatter tuple
@@ -193,17 +186,15 @@
      *
      * @param diskComponents
      *            the components to be replicated
-     * @param bulkload
-     *            true if the components were bulkloaded, false otherwise
      * @param opType
      *            the operation type
      * @throws HyracksDataException
      */
-    void scheduleReplication(List<ILSMDiskComponent> diskComponents, boolean bulkload, LSMOperationType opType)
+    void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
             throws HyracksDataException;
 
     /**
-     * Force a flush of the in-memory component.
+     * Flush an in-memory component.
      *
      * @throws HyracksDataException
      * @throws TreeIndexException
@@ -265,9 +256,10 @@
     void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException;
 
     /**
-    * Update the filter of an LSM index
-    * @param tuple
-    * @throws HyracksDataException
-    */
+     * Update the filter of an LSM index
+     *
+     * @param tuple
+     * @throws HyracksDataException
+     */
     void updateFilter(ITupleReference tuple) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
index b34b403..238d5f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -19,12 +19,11 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -94,26 +93,26 @@
     void setRecovery(boolean recovery);
 
     /**
-     * @return the IO Operation type associated with this context
+     * @return the IO operation associated with this context
      */
-    LSMIOOperationType getIoOperationType();
+    ILSMIOOperation getIoOperation();
 
     /**
-     * Set the IO Operation type associated with this context
+     * Set the IO operation associated with this context
      *
-     * @param ioOpType
+     * @param ioOperation
      */
-    void setIoOperationType(LSMIOOperationType ioOpType);
+    void setIoOperation(ILSMIOOperation ioOperation);
 
     /**
-     * @return the new component produced by this operation if any, null otherwise
-     */
-    ILSMDiskComponent getNewComponent();
-
-    /**
-     * Set the new component produced by this operation
+     * Set a map in the context to pass pairs of keys and values
      *
-     * @param component
+     * @param map
      */
-    void setNewComponent(ILSMDiskComponent component);
+    void setParameters(Map<String, Object> map);
+
+    /**
+     * @return the key value map of the context
+     */
+    Map<String, Object> getParameters();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
index 4ff6377..c86f7b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
@@ -63,19 +63,6 @@
     void setModified();
 
     /**
-     * request the component to be active
-     */
-    void requestActivation();
-
-    /**
-     * Set the component state
-     *
-     * @param state
-     *            the new state
-     */
-    void setState(ComponentState state);
-
-    /**
      * Allocates memory to this component, create and activate it.
      * This method is atomic. If an exception is thrown, then the call had no effect.
      *
@@ -108,8 +95,14 @@
      *
      * @param newId
      * @param force
-     *      Whether to force reset the Id to skip sanity checks
+     *            Whether to force reset the Id to skip sanity checks
      * @throws HyracksDataException
      */
     void resetId(ILSMComponentId newId, boolean force) throws HyracksDataException;
+
+    /**
+     * Set the component state to be unwritable to prevent future writers from non-force
+     * entry to the component
+     */
+    void setUnwritable();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
index fc1d5f4..74a5d60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -30,8 +31,8 @@
      * This function is used to create a BulkLoader for a transaction that is capable of insertions and deletions
      * and the bulk loaded component is hidden from the index
      */
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException;
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException;
 
     /**
      * This function is used to commit the previous transaction if it was resulted in creating any components
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
new file mode 100644
index 0000000..e5ba81e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+public interface IoOperationCompleteListener {
+
+    /**
+     * Called when an IO operation completes
+     *
+     * @param operation
+     */
+    void completed(ILSMIOOperation operation);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
index 63d2697..e200bfd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
@@ -22,6 +22,7 @@
     SEARCH,
     MODIFICATION,
     FORCE_MODIFICATION,
+    LOAD,
     FLUSH,
     MERGE,
     REPLICATE,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
index 4e8cf71..682ffef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
@@ -57,7 +57,7 @@
         indexHelper.open();
         ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFullMerge(index.getIOOperationCallback());
+        accessor.scheduleFullMerge();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
index aee46f0..fc9a362 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
@@ -18,11 +18,18 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
 
 public abstract class AbstractIoOperation implements ILSMIOOperation {
 
@@ -30,6 +37,11 @@
     protected final FileReference target;
     protected final ILSMIOOperationCallback callback;
     protected final String indexIdentifier;
+    private Throwable failure;
+    private LSMIOOperationStatus status = LSMIOOperationStatus.SUCCESS;
+    private ILSMDiskComponent newComponent;
+    private boolean completed = false;
+    private List<IoOperationCompleteListener> completeListeners;
 
     public AbstractIoOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
             String indexIdentifier) {
@@ -63,4 +75,75 @@
     public String getIndexIdentifier() {
         return indexIdentifier;
     }
+
+    @Override
+    public Throwable getFailure() {
+        return failure;
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        status = LSMIOOperationStatus.FAILURE;
+        this.failure = ExceptionUtils.suppress(this.failure, failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        this.status = status;
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return newComponent;
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        this.newComponent = component;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (completed) {
+            throw new IllegalStateException("Multiple destroy calls");
+        }
+        callback.completed(this);
+        completed = true;
+        if (completeListeners != null) {
+            for (IoOperationCompleteListener listener : completeListeners) {
+                listener.completed(this);
+            }
+            completeListeners = null;
+        }
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void sync() throws InterruptedException {
+        while (!completed) {
+            wait();
+        }
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return accessor.getOpContext().getParameters();
+    }
+
+    @Override
+    public synchronized void addCompleteListener(IoOperationCompleteListener listener) {
+        if (completed) {
+            listener.completed(this);
+        } else {
+            if (completeListeners == null) {
+                completeListeners = new LinkedList<>();
+            }
+            completeListeners.add(listener);
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
index 84d2fe5..574a371 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
@@ -51,4 +51,9 @@
     public final AbstractLSMIndex getLsmIndex() {
         return lsmIndex;
     }
+
+    @Override
+    public int getReaderCount() {
+        return readerCount;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index c4616d3..aa312fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -24,6 +24,8 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
@@ -52,27 +54,34 @@
     }
 
     @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        if (ioOperationType != LSMIOOperationType.MERGE) {
+            throw new IllegalStateException("Unsupported operation type: " + ioOperationType);
+        }
+        if (state == ComponentState.INACTIVE) {
+            throw new IllegalStateException("Trying to schedule a merge of an inactive disk component");
+        }
+        if (state == ComponentState.READABLE_MERGING) {
+            // This should never happen unless there are two concurrent merges that were scheduled
+            // concurrently and they have interleaving components to be merged.
+            // This should be handled properly by the merge policy, but we guard against that here anyway.
+            throw new IllegalStateException("The disk component has already been scheduled for a merge");
+        }
+        state = ComponentState.READABLE_MERGING;
+    }
+
+    @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
         if (state == ComponentState.INACTIVE) {
             throw new IllegalStateException("Trying to enter an inactive disk component");
         }
-
         switch (opType) {
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
-                readerCount++;
-                break;
             case MERGE:
-                if (state == ComponentState.READABLE_MERGING) {
-                    // This should never happen unless there are two concurrent merges that were scheduled
-                    // concurrently and they have interleaving components to be merged.
-                    // This should be handled properly by the merge policy, but we guard against that here anyway.
-                    return false;
-                }
-                state = ComponentState.READABLE_MERGING;
                 readerCount++;
                 break;
             default:
@@ -86,19 +95,22 @@
             throws HyracksDataException {
         switch (opType) {
             case MERGE:
+                readerCount--;
                 // In case two merge operations were scheduled to merge an overlapping set of components,
                 // the second merge will fail and it must reset those components back to their previous state.
                 if (failedOperation) {
                     state = ComponentState.READABLE_UNWRITABLE;
+                } else {
+                    state = (readerCount == 0) ? ComponentState.INACTIVE : ComponentState.UNREADABLE_UNWRITABLE;
                 }
-                // Fallthrough
+                break;
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
                 readerCount--;
-                if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+                if (readerCount == 0 && state == ComponentState.UNREADABLE_UNWRITABLE) {
                     state = ComponentState.INACTIVE;
                 }
                 break;
@@ -163,9 +175,15 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        getIndex().deactivate();
-        getIndex().destroy();
+    public final void deactivateAndDestroy() throws HyracksDataException {
+        deactivateAndPurge();
+        destroy();
+    }
+
+    @Override
+    public final void deactivateAndPurge() throws HyracksDataException {
+        deactivate();
+        purge();
     }
 
     @Override
@@ -178,9 +196,7 @@
         getIndex().deactivate();
     }
 
-    @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        getIndex().deactivate();
+    protected void purge() throws HyracksDataException {
         getIndex().purge();
     }
 
@@ -203,17 +219,27 @@
                 getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
     }
 
+    /**
+     * Allows sub-class extend this method to use specialized bulkloader for merge
+     */
+    protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        return this.createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+    }
+
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
         ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
-                new ChainedLSMDiskComponentBulkLoader(this, cleanupEmptyComponent);
+                new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
         if (withFilter && getLsmIndex().getFilterFields() != null) {
             chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
         }
-        chainedBulkLoader
-                .addBulkLoader(createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+        IChainedComponentBulkLoader indexBulkloader = operation.getIOOpertionType() == LSMIOOperationType.MERGE
+                ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex)
+                : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        chainedBulkLoader.addBulkLoader(indexBulkloader);
         return chainedBulkLoader;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 2db2f79..377a218 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -21,9 +21,11 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -41,6 +43,7 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
@@ -74,7 +77,6 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final ILSMHarness lsmHarness;
     protected final IIOManager ioManager;
-    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ILSMIOOperationCallback ioOpCallback;
 
     // In-memory components.
@@ -95,6 +97,7 @@
     protected final int[] filterFields;
     protected final boolean durable;
     protected boolean isActive;
+    protected volatile boolean isDeactivating = false;
     protected final AtomicBoolean[] flushRequests;
     protected volatile boolean memoryComponentsAllocated = false;
     protected ITracer tracer;
@@ -102,6 +105,7 @@
     protected final ILSMDiskComponentFactory componentFactory;
     // Factory for creating on-disk index components during bulkload.
     protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
+    private int numScheduledFlushes = 0;
 
     public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
@@ -115,7 +119,6 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
@@ -127,11 +130,12 @@
         this.inactiveDiskComponents = new LinkedList<>();
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new LSMHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(), tracer);
+        lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
+                tracer);
         isActive = false;
         diskComponents = new ArrayList<>();
         memoryComponents = new ArrayList<>();
-        currentMutableComponentId = new AtomicInteger();
+        currentMutableComponentId = new AtomicInteger(ioOpCallbackFactory.getCurrentMemoryComponentIndex());
         flushRequests = new AtomicBoolean[virtualBufferCaches.size()];
         for (int i = 0; i < virtualBufferCaches.size(); i++) {
             flushRequests[i] = new AtomicBoolean();
@@ -148,13 +152,13 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled());
+        lsmHarness = new ExternalIndexHarness(this, ioScheduler, mergePolicy, opTracker,
+                diskBufferCache.isReplicationEnabled());
         isActive = false;
         diskComponents = new LinkedList<>();
         this.inactiveDiskComponents = new LinkedList<>();
@@ -205,28 +209,35 @@
         deactivate(true);
     }
 
+    @SuppressWarnings({ "squid:S1181", "squid:S2142" })
     @Override
     public synchronized void deactivate(boolean flush) throws HyracksDataException {
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_INDEX);
         }
-        if (flush) {
-            flushMemoryComponent();
-        }
-        deactivateDiskComponents();
-        deallocateMemoryComponents();
-        isActive = false;
-    }
-
-    private void flushMemoryComponent() throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
-        ILSMIndexAccessor accessor = createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFlush(cb);
+        // The following member is used to prevent scheduling of new merges as memory components
+        // get flushed. This now works only if the caller of deactivate waited for all IO
+        // operations to complete. Otherwise, disk components can be evicted while background
+        // merges are ongoing.
+        isDeactivating = true;
         try {
-            cb.waitForIO();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. STARTED", this);
+            if (flush && memoryComponentsAllocated) {
+                try {
+                    createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFlush().sync();
+                } catch (InterruptedException e) {
+                    throw HyracksDataException.create(e);
+                }
+                LOGGER.log(Level.INFO, "Deactivating the index: {}. Flushed", this);
+            }
+            LOGGER.log(Level.INFO, "Deactivating the disk components of: {}", this);
+            deactivateDiskComponents();
+            LOGGER.log(Level.INFO, "Deallocating memory components of: {}", this);
+            deallocateMemoryComponents();
+            isActive = false;
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. COMPLETED", this);
+        } finally {
+            isDeactivating = false;
         }
     }
 
@@ -282,6 +293,8 @@
                 c.reset();
             }
         }
+        numScheduledFlushes = 0;
+        currentMutableComponentId.set(0);
     }
 
     @Override
@@ -297,19 +310,18 @@
         switch (ctx.getOperation()) {
             case UPDATE:
             case PHYSICALDELETE:
-            case FLUSH:
-            case DELETE_MEMORY_COMPONENT:
+            case DELETE_COMPONENTS:
             case DELETE:
             case UPSERT:
                 operationalComponents.add(memoryComponents.get(cmc));
                 break;
             case INSERT:
-                addOperationalMutableComponents(operationalComponents, true);
+                addOperationalMemoryComponents(operationalComponents, true);
                 operationalComponents.addAll(diskComponents);
                 break;
             case SEARCH:
                 if (memoryComponentsAllocated) {
-                    addOperationalMutableComponents(operationalComponents, false);
+                    addOperationalMemoryComponents(operationalComponents, false);
                 }
                 if (filterManager != null) {
                     for (int i = 0; i < diskComponents.size(); i++) {
@@ -326,13 +338,6 @@
                 }
 
                 break;
-            case MERGE:
-            case DELETE_DISK_COMPONENTS:
-                operationalComponents.addAll(ctx.getComponentsToBeMerged());
-                break;
-            case FULL_MERGE:
-                operationalComponents.addAll(diskComponents);
-                break;
             case REPLICATE:
                 operationalComponents.addAll(ctx.getComponentsToBeReplicated());
                 break;
@@ -350,65 +355,169 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(ctx.getComponentHolder());
-        ILSMIOOperation flushOp = createFlushOperation(opCtx, componentFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(flushOp, tracer));
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMMemoryComponent flushingComponent = getCurrentMemoryComponent();
+        if (flushingComponent.getWriterCount() > 0) {
+            throw new IllegalStateException(
+                    "createFlushOperation is called on a component with writers: " + flushingComponent);
+        }
+        // take care of the flush cycling
+        ILSMIOOperation flushOp =
+                TracedIOOperation.wrap(createFlushOperation(createOpContext(NoOpIndexAccessParameters.INSTANCE),
+                        fileManager.getRelFlushFileReference(), ioOpCallback), tracer);
+        // Changing the flush status should *always* precede changing the mutable component.
+        flushingComponent.schedule(LSMIOOperationType.FLUSH);
+        numScheduledFlushes++;
+        changeFlushStatusForCurrentMutableCompoent(false);
+        changeMutableComponent();
+        ILSMIndexAccessor accessor = flushOp.getAccessor();
+        ILSMIndexOperationContext flushCtx = accessor.getOpContext();
+        flushCtx.setOperation(ctx.getOperation()); // Could be component delete
+        flushCtx.getComponentHolder().add(flushingComponent);
+        flushCtx.setIoOperation(flushOp);
+        propagateMap(ctx, flushCtx);
+        ioOpCallback.scheduled(flushOp);
+        return flushOp;
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
-        // merge must create a different op ctx
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(mergingComponents);
-        mergingComponents.stream().map(ILSMDiskComponent.class::cast).forEach(opCtx.getComponentsToBeMerged()::add);
-        ILSMDiskComponent firstComponent = (ILSMDiskComponent) mergingComponents.get(0);
-        ILSMDiskComponent lastComponent = (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1);
-        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
-        ILSMIOOperation mergeOp = createMergeOperation(opCtx, mergeFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(mergeOp, tracer));
-    }
-
-    private void addOperationalMutableComponents(List<ILSMComponent> operationalComponents, boolean modification) {
-        int cmc = currentMutableComponentId.get();
-        int numMutableComponents = memoryComponents.size();
-        for (int i = 0; i < numMutableComponents - 1; i++) {
-            ILSMMemoryComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
-            if (c.isReadable()) {
-                // Make sure newest components are added first if readable
-                operationalComponents.add(0, c);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        List<ILSMDiskComponent> mergingComponents = ctx.getComponentsToBeMerged();
+        // Merge operation can fail if another merge is already scheduled on those components
+        // This should be guarded against by the merge policy but we still protect against here
+        if (isDeactivating
+                || (mergingComponents.size() < 2 && ctx.getOperation() != IndexOperation.DELETE_COMPONENTS)) {
+            return NoOpIoOperation.INSTANCE;
+        }
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            if (mergingComponents.get(i).getState() == ComponentState.READABLE_MERGING) {
+                return NoOpIoOperation.INSTANCE;
             }
         }
-        // The current mutable component is added if modification operation or if readable
-        // This ensures that activation of new component only happens in case of modifications
-        // and allow for controlling that without stopping search operations
-        ILSMMemoryComponent c = memoryComponents.get(cmc);
-        if (modification || c.isReadable()) {
-            operationalComponents.add(0, c);
+        // merge must create a different op ctx
+        AbstractLSMIndexOperationContext mergeCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
+        mergeCtx.setOperation(ctx.getOperation());
+        mergeCtx.getComponentHolder().addAll(mergingComponents);
+        propagateMap(ctx, mergeCtx);
+        mergingComponents.stream().forEach(mergeCtx.getComponentsToBeMerged()::add);
+        ILSMDiskComponent firstComponent = mergingComponents.get(0);
+        ILSMDiskComponent lastComponent = mergingComponents.get(mergingComponents.size() - 1);
+        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
+        ILSMIOOperation mergeOp =
+                TracedIOOperation.wrap(createMergeOperation(mergeCtx, mergeFileRefs, ioOpCallback), tracer);
+        mergeCtx.setIoOperation(mergeOp);
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            mergingComponents.get(i).schedule(LSMIOOperationType.MERGE);
         }
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+    }
+
+    private static void propagateMap(ILSMIndexOperationContext src, ILSMIndexOperationContext destination) {
+        Map<String, Object> map = src.getParameters();
+        if (map != null && !map.isEmpty()) {
+            destination.setParameters(new HashMap<>(map));
+        }
+    }
+
+    private void addOperationalMemoryComponents(List<ILSMComponent> operationalComponents, boolean modification) {
+        // add current memory component first if needed
+        if (numScheduledFlushes < memoryComponents.size()) {
+            ILSMMemoryComponent c = memoryComponents.get(currentMutableComponentId.get());
+            // The current mutable component is added if modification or readable
+            // This ensures that activation of new component only happens in case of modifications
+            // and allow for controlling that without stopping search operations
+            if (modification || c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+        if (modification && numScheduledFlushes >= memoryComponents.size()) {
+            // will fail the enterComponent call and retry
+            operationalComponents.add(memoryComponents.get(0));
+            return;
+        }
+        addImmutableMemoryComponents(operationalComponents);
+    }
+
+    private void addImmutableMemoryComponents(List<ILSMComponent> operationalComponents) {
+        int cmc = currentMutableComponentId.get();
+        int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+        int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+        for (int i = 0; i < numImmutableMemoryComponents; i++) {
+            next--;
+            if (next < 0) {
+                next = memoryComponents.size() - 1;
+            }
+            //newer components first
+            ILSMMemoryComponent c = memoryComponents.get(next);
+            if (c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+    }
+
+    private ILSMMemoryComponent getOldestReadableMemoryComponent() {
+        synchronized (getOperationTracker()) {
+            int cmc = currentMutableComponentId.get();
+            int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+            int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                next--;
+                if (next < 0) {
+                    next = memoryComponents.size() - 1;
+                }
+            }
+
+            // start going forward
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                if (memoryComponents.get(next).isReadable()) {
+                    return memoryComponents.get(next);
+                }
+                next++;
+                if (next == memoryComponents.size()) {
+                    next = 0;
+                }
+            }
+            throw new IllegalStateException("Couldn't find any readable component");
+        }
+    }
+
+    private int getNextToBeFlushed() {
+        // we have:
+        // 1. currentMemeoryComponent
+        // 2. numMemoryComponents
+        // 3. numScheduledFlushes
+        int diff = numScheduledFlushes % memoryComponents.size();
+        int cmc = currentMutableComponentId.get() - diff;
+        return cmc < 0 ? memoryComponents.size() + cmc : cmc;
     }
 
     @Override
     public final IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex) throws HyracksDataException {
+        return createBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, null);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException {
         if (checkIfEmptyIndex && !isEmptyIndex()) {
             throw HyracksDataException.create(ErrorCode.LOAD_NON_EMPTY_INDEX);
         }
-        return createBulkLoader(fillLevel, verifyInput, numElementsHint);
+        return createBulkLoader(fillFactor, verifyInput, numElementsHint, parameters);
     }
 
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
         AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setIoOperationType(LSMIOOperationType.LOAD);
-        ioOpCallback.beforeOperation(opCtx);
+        opCtx.setParameters(parameters);
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        LoadOperation loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+        loadOp.setNewComponent(createDiskComponent(bulkLoadComponentFactory,
+                componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
+                componentFileRefs.getBloomFilterFileReference(), true));
+        ioOpCallback.scheduled(loadOp);
+        opCtx.setIoOperation(loadOp);
         return new LSMIndexDiskComponentBulkLoader(this, opCtx, fillLevel, verifyInput, numElementsHint);
     }
 
@@ -503,7 +612,6 @@
     @Override
     public void changeMutableComponent() {
         currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
-        memoryComponents.get(currentMutableComponentId.get()).requestActivation();
     }
 
     @Override
@@ -527,11 +635,6 @@
     }
 
     @Override
-    public ILSMIOOperationScheduler getIOScheduler() {
-        return ioScheduler;
-    }
-
-    @Override
     public ILSMIOOperationCallback getIOOperationCallback() {
         return ioOpCallback;
     }
@@ -556,8 +659,9 @@
     @Override
     public final String toString() {
         return "{\"class\" : \"" + getClass().getSimpleName() + "\", \"dir\" : \"" + fileManager.getBaseDir()
-                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents.size()) + ", \"disk\" : "
-                + diskComponents.size() + "}";
+                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents) + ", \"disk\" : "
+                + diskComponents.size() + ", \"num-scheduled-flushes\":" + numScheduledFlushes
+                + ", \"current-memory-component\":" + currentMutableComponentId.get() + "}";
     }
 
     @Override
@@ -567,8 +671,12 @@
 
     @Override
     public boolean isCurrentMutableComponentEmpty() throws HyracksDataException {
-        //check if the current memory component has been modified
-        return !memoryComponents.get(currentMutableComponentId.get()).isModified();
+        synchronized (getOperationTracker()) {
+            ILSMMemoryComponent cmc = getCurrentMemoryComponent();
+            ComponentState state = cmc.getState();
+            return state == ComponentState.READABLE_UNWRITABLE_FLUSHING || state == ComponentState.INACTIVE
+                    || state == ComponentState.UNREADABLE_UNWRITABLE || !cmc.isModified();
+        }
     }
 
     @Override
@@ -583,7 +691,7 @@
 
     @Override
     public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
-            boolean bulkload, ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
+            ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
         //get set of files to be replicated for this component
         Set<String> componentFiles = new HashSet<>();
 
@@ -593,7 +701,7 @@
         }
 
         ReplicationExecutionType executionType;
-        if (bulkload) {
+        if (opType == LSMOperationType.LOAD) {
             executionType = ReplicationExecutionType.SYNC;
         } else {
             executionType = ReplicationExecutionType.ASYNC;
@@ -700,40 +808,57 @@
     }
 
     @Override
+    public void resetCurrentComponentIndex() {
+        synchronized (lsmHarness.getOperationTracker()) {
+            // validate no reader in any of the memory components and that all of them are INVALID
+            for (ILSMMemoryComponent c : memoryComponents) {
+                if (c.getReaderCount() > 0) {
+                    throw new IllegalStateException(
+                            "Attempt to reset current component index while readers are inside the components. " + c);
+                }
+                if (c.getState() != ComponentState.INACTIVE) {
+                    throw new IllegalStateException(
+                            "Attempt to reset current component index while a component is not INACTIVE. " + c);
+                }
+            }
+            currentMutableComponentId.set(0);
+            memoryComponents.get(0);
+            try {
+                memoryComponents.get(0).resetId(null, true);
+            } catch (HyracksDataException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+    }
+
+    @Override
     public final ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        if (opCtx.getOperation() == IndexOperation.DELETE_MEMORY_COMPONENT) {
+        ILSMMemoryComponent memoryComponent = (ILSMMemoryComponent) opCtx.getComponentHolder().get(0);
+        if (memoryComponent != getOldestReadableMemoryComponent()) {
+            throw new IllegalStateException("An attempt to flush a memory component that is not the oldest");
+        }
+        if (!memoryComponent.isModified() || opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS) {
             return EmptyComponent.INSTANCE;
         }
         if (LOGGER.isInfoEnabled()) {
             FlushOperation flushOp = (FlushOperation) operation;
-            LOGGER.log(Level.INFO, "Flushing component with id: " + flushOp.getFlushingComponent().getId());
+            LOGGER.log(Level.INFO,
+                    "Flushing component with id: " + flushOp.getFlushingComponent().getId() + " in the index " + this);
         }
-        try {
-            return doFlush(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute flush " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return doFlush(operation);
     }
 
     @Override
     public final ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        try {
-            return opCtx.getOperation() == IndexOperation.DELETE_DISK_COMPONENTS ? EmptyComponent.INSTANCE
-                    : doMerge(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute merge " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS ? EmptyComponent.INSTANCE : doMerge(operation);
     }
 
-    protected void cleanUpFiles(ILSMIOOperation operation, Exception e) {
+    @Override
+    public void cleanUpFilesForFailedOperation(ILSMIOOperation operation) {
         LSMComponentFileReferences componentFiles = operation.getComponentFiles();
         if (componentFiles == null) {
             return;
@@ -742,14 +867,25 @@
         for (FileReference file : files) {
             try {
                 if (file != null) {
+                    diskBufferCache.closeFileIfOpen(file);
                     diskBufferCache.deleteFile(file);
                 }
-            } catch (HyracksDataException hde) {
-                e.addSuppressed(hde);
+            } catch (Throwable th) { // NOSONAR Must catch all failures
+                operation.getFailure().addSuppressed(th);
             }
         }
     }
 
+    @Override
+    public String getIndexIdentifier() {
+        return fileManager.getBaseDir().getAbsolutePath();
+    }
+
+    //Called when a memory component is reset
+    public void memoryComponentsReset() {
+        numScheduledFlushes = Integer.max(0, numScheduledFlushes - 1);
+    }
+
     protected abstract LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException;
 
@@ -765,5 +901,4 @@
     protected abstract ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException;
 
     protected abstract ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException;
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
index c993874..2e133a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -28,7 +29,7 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -57,8 +58,8 @@
     private long enterExitTime = 0L;
     protected boolean skipFilter = false;
     protected boolean recovery = false;
-    private LSMIOOperationType ioOpType = LSMIOOperationType.NOOP;
-    private ILSMDiskComponent newDiskComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> parametersMap;
 
     public AbstractLSMIndexOperationContext(ILSMIndex index, int[] treeFields, int[] filterFields,
             IBinaryComparatorFactory[] filterCmpFactories, ISearchOperationCallback searchCallback,
@@ -210,22 +211,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOpType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOpType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newDiskComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.parametersMap = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newDiskComponent = component;
+    public Map<String, Object> getParameters() {
+        return parametersMap;
     }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 9596495..ab00dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -39,31 +40,56 @@
     private final IVirtualBufferCache vbc;
     private final AtomicBoolean isModified;
     private int writerCount;
-    private boolean requestedToBeActive;
+    private int pendingFlushes = 0;
     private final MemoryComponentMetadata metadata;
     private ILSMComponentId componentId;
 
-    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
+    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
         super(lsmIndex, filter);
         this.vbc = vbc;
         writerCount = 0;
-        if (isActive) {
-            state = ComponentState.READABLE_WRITABLE;
-        } else {
-            state = ComponentState.INACTIVE;
-        }
+        state = ComponentState.INACTIVE;
         isModified = new AtomicBoolean();
         metadata = new MemoryComponentMetadata();
     }
 
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        activeate();
+        if (ioOperationType == LSMIOOperationType.FLUSH) {
+            if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to schedule a flush when writerCount != 0");
+                }
+                state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+            } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
+                    || state == ComponentState.UNREADABLE_UNWRITABLE) {
+                // There is an ongoing flush. Increase pending flush count
+                pendingFlushes++;
+            } else {
+                throw new IllegalStateException("Trying to schedule a flush when the component state = " + state);
+            }
+        } else {
+            throw new UnsupportedOperationException("Unsupported operation " + ioOperationType);
+        }
+    }
+
+    private void activeate() throws HyracksDataException {
+        if (state == ComponentState.INACTIVE) {
+            state = ComponentState.READABLE_WRITABLE;
+            lsmIndex.getIOOperationCallback().recycled(this);
+        }
+    }
+
     @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException {
-        if (state == ComponentState.INACTIVE && requestedToBeActive) {
-            state = ComponentState.READABLE_WRITABLE;
-            requestedToBeActive = false;
-            lsmIndex.getIOOperationCallback().recycled(this, true);
-        }
+        activeate();
         switch (opType) {
             case FORCE_MODIFICATION:
                 if (isMutableComponent) {
@@ -97,7 +123,6 @@
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE
                         || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
@@ -107,16 +132,18 @@
                 }
                 break;
             case FLUSH:
-                if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
-                    if (writerCount != 0) {
-                        throw new IllegalStateException("Trying to flush when writerCount != 0");
-                    }
-                    state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
-                    readerCount++;
-                } else {
+                if (state == ComponentState.UNREADABLE_UNWRITABLE) {
                     return false;
                 }
-                break;
+                if (state != ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+                    throw new IllegalStateException("Trying to flush when component state = " + state);
+                }
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to flush when writerCount = " + writerCount);
+                }
+                readerCount++;
+                return true;
+
             default:
                 throw new UnsupportedOperationException("Unsupported operation " + opType);
         }
@@ -139,15 +166,14 @@
                 } else {
                     readerCount--;
                     if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                        state = ComponentState.INACTIVE;
+                        reset();
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 readerCount--;
                 if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 }
                 break;
             case FLUSH:
@@ -156,12 +182,12 @@
                 }
                 readerCount--;
                 if (failedOperation) {
-                    // if flush failed, return the component state to READABLE_UNWRITABLE
-                    state = ComponentState.READABLE_UNWRITABLE;
+                    // If flush failed, keep the component state to READABLE_UNWRITABLE_FLUSHING
                     return;
                 }
+                // operation succeeded
                 if (readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 } else {
                     state = ComponentState.UNREADABLE_UNWRITABLE;
                 }
@@ -177,20 +203,15 @@
 
     @Override
     public boolean isReadable() {
-        if (state == ComponentState.INACTIVE || state == ComponentState.UNREADABLE_UNWRITABLE) {
-            return false;
+        return state != ComponentState.INACTIVE && state != ComponentState.UNREADABLE_UNWRITABLE;
+    }
+
+    @Override
+    public void setUnwritable() {
+        if (state != ComponentState.READABLE_WRITABLE) {
+            throw new IllegalStateException("Attempt to set unwritable a component that is " + state);
         }
-        return true;
-    }
-
-    @Override
-    public void setState(ComponentState state) {
-        this.state = state;
-    }
-
-    @Override
-    public void requestActivation() {
-        requestedToBeActive = true;
+        this.state = ComponentState.READABLE_UNWRITABLE;
     }
 
     @Override
@@ -210,12 +231,23 @@
 
     @Override
     public final void reset() throws HyracksDataException {
+        state = ComponentState.INACTIVE;
         isModified.set(false);
         metadata.reset();
         if (filter != null) {
             filter.reset();
         }
         doReset();
+        lsmIndex.memoryComponentsReset();
+        // a flush can be pending on a component that just completed its flush... here is when this can happen:
+        // primary index has 2 components, secondary index has 2 components.
+        // 2 flushes are scheduled on each p1, p2, s1, and s2.
+        // p1 and p2 both finish. primary component 1 gets full and secondary doesn't have any entries (optional field).
+        // then flush is scheduled on p1, s1 will have a pending flush in that case.
+        if (pendingFlushes > 0) {
+            schedule(LSMIOOperationType.FLUSH);
+            pendingFlushes--;
+        }
     }
 
     protected void doReset() throws HyracksDataException {
@@ -267,6 +299,7 @@
     @Override
     public final void deallocate() throws HyracksDataException {
         try {
+            state = ComponentState.INACTIVE;
             doDeallocate();
         } finally {
             getIndex().getBufferCache().close();
@@ -297,7 +330,7 @@
 
     @Override
     public void resetId(ILSMComponentId componentId, boolean force) throws HyracksDataException {
-        if (!force && this.componentId != null && !componentId.missing() // for backward compatibility
+        if (!force && this.componentId != null
                 && this.componentId.compareTo(componentId) != IdCompareResult.LESS_THAN) {
             throw new IllegalStateException(
                     this + " receives illegal id. Old id " + this.componentId + ", new id " + componentId);
@@ -306,6 +339,15 @@
             LOGGER.log(Level.INFO, "Component Id was reset from " + this.componentId + " to " + componentId);
         }
         this.componentId = componentId;
-        LSMComponentIdUtils.persist(this.componentId, metadata);
+        if (componentId != null) {
+            LSMComponentIdUtils.persist(this.componentId, metadata);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "{\"class\":\"" + getClass().getSimpleName() + "\", \"state\":\"" + state + "\", \"writers\":"
+                + writerCount + ", \"readers\":" + readerCount + ", \"pendingFlushes\":" + pendingFlushes
+                + ", \"id\":\"" + componentId + "\"}";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
new file mode 100644
index 0000000..373d7e7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+
+public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
+
+    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
+    }
+
+    public abstract AbstractTreeIndex getBuddyIndex();
+
+    @Override
+    public void doReset() throws HyracksDataException {
+        super.doReset();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doAllocate() throws HyracksDataException {
+        super.doAllocate();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doDeallocate() throws HyracksDataException {
+        super.doDeallocate();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        super.validate();
+        getBuddyIndex().validate();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index a439ace..e4b845a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -18,94 +18,80 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 
-public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+public class AsynchronousScheduler implements ILSMIOOperationScheduler, Closeable {
     // Since this is a asynchronous scheduler, we make sure that flush operations coming from the same lsm index
     // will be executed serially in same order of scheduling the operations. Look at asterix issue 630.
 
-    public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
-    private ExecutorService executor;
-    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<String, ILSMIOOperation>();
-    private final Map<String, PriorityQueue<ILSMIOOperation>> waitingFlushOperations =
-            new HashMap<String, PriorityQueue<ILSMIOOperation>>();
+    private final ExecutorService executor;
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations = new HashMap<>();
+    private final Map<String, Throwable> failedGroups = new HashMap<>();
 
-    public void init(ThreadFactory threadFactory) {
-        // Creating an executor with the same configuration of Executors.newCachedThreadPool.
-        executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
-                threadFactory) {
-
-            @Override
-            protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
-                return new LSMIOOperationTask<T>(callable);
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            protected void afterExecute(Runnable r, Throwable t) {
-                super.afterExecute(r, t);
-                LSMIOOperationTask<Boolean> task = (LSMIOOperationTask<Boolean>) r;
-                ILSMIOOperation executedOp = task.getOperation();
-                if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-                    String id = executedOp.getIndexIdentifier();
-                    synchronized (this) {
-                        runningFlushOperations.remove(id);
-                        if (waitingFlushOperations.containsKey(id)) {
-                            try {
-                                ILSMIOOperation op = waitingFlushOperations.get(id).poll();
-                                if (op != null) {
-                                    scheduleOperation(op);
-                                } else {
-                                    waitingFlushOperations.remove(id);
-                                }
-                            } catch (HyracksDataException e) {
-                                t = e.getCause();
-                            }
-                        }
-                    }
-                }
-            }
-        };
+    public AsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback) {
+        executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations,
+                waitingFlushOperations, failedGroups);
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-        if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
-            executor.submit(operation);
-        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-            String id = operation.getIndexIdentifier();
-            synchronized (executor) {
-                if (runningFlushOperations.containsKey(id)) {
-                    if (waitingFlushOperations.containsKey(id)) {
-                        waitingFlushOperations.get(id).offer(operation);
-                    } else {
-                        PriorityQueue<ILSMIOOperation> q = new PriorityQueue<ILSMIOOperation>();
-                        q.offer(operation);
-                        waitingFlushOperations.put(id, q);
-                    }
-                } else {
-                    runningFlushOperations.put(id, operation);
-                    executor.submit(operation);
-                }
-            }
-        } else {
-            // this should never happen
-            // just guard here to avoid silient failures in case of future extensions
-            throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
+    public void scheduleOperation(ILSMIOOperation operation) {
+        switch (operation.getIOOpertionType()) {
+            case FLUSH:
+                scheduleFlush(operation);
+                break;
+            case MERGE:
+                executor.submit(operation);
+                break;
+            case NOOP:
+                return;
+            default:
+                // this should never happen
+                // just guard here to avoid silent failures in case of future extensions
+                throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
         }
     }
+
+    private void scheduleFlush(ILSMIOOperation operation) {
+        String id = operation.getIndexIdentifier();
+        synchronized (executor) {
+            if (failedGroups.containsKey(id)) {
+                // Group failure. Fail the operation right away
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                        failedGroups.get(id)));
+                operation.complete();
+                return;
+            }
+            if (runningFlushOperations.containsKey(id)) {
+                if (waitingFlushOperations.containsKey(id)) {
+                    waitingFlushOperations.get(id).offer(operation);
+                } else {
+                    Deque<ILSMIOOperation> q = new ArrayDeque<>();
+                    q.offer(operation);
+                    waitingFlushOperations.put(id, q);
+                }
+            } else {
+                runningFlushOperations.put(id, operation);
+                executor.submit(operation);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        executor.shutdown();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
deleted file mode 100644
index a8ee286..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-public class BlockingIOOperationCallbackWrapper implements ILSMIOOperationCallback {
-
-    private boolean notified = false;
-
-    private final ILSMIOOperationCallback wrappedCallback;
-
-    public BlockingIOOperationCallbackWrapper(ILSMIOOperationCallback callback) {
-        this.wrappedCallback = callback;
-    }
-
-    public synchronized void waitForIO() throws InterruptedException {
-        while (!notified) {
-            wait();
-        }
-        notified = false;
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.beforeOperation(opCtx);
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterOperation(opCtx);
-    }
-
-    @Override
-    public synchronized void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterFinalize(opCtx);
-        notifyAll();
-        notified = true;
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        wrappedCallback.recycled(component, componentSwitched);
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        wrappedCallback.allocated(component);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
index f38614c..6e0606a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -18,26 +18,32 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import java.util.LinkedList;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.util.annotations.CriticalPath;
 
 /**
  * Class encapsulates a chain of operations, happening during an LSM disk component bulkload
  */
 public class ChainedLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
 
-    private List<IChainedComponentBulkLoader> bulkloaderChain = new LinkedList<>();
-    private boolean isEmptyComponent = true;
-    private boolean cleanedUpArtifacts = false;
+    private List<IChainedComponentBulkLoader> bulkloaderChain = new ArrayList<>();
+    private final ILSMIOOperation operation;
     private final ILSMDiskComponent diskComponent;
     private final boolean cleanupEmptyComponent;
+    private boolean isEmptyComponent = true;
+    private boolean cleanedUpArtifacts = false;
 
-    public ChainedLSMDiskComponentBulkLoader(ILSMDiskComponent diskComponent, boolean cleanupEmptyComponent) {
+    public ChainedLSMDiskComponentBulkLoader(ILSMIOOperation operation, ILSMDiskComponent diskComponent,
+            boolean cleanupEmptyComponent) {
+        this.operation = operation;
         this.diskComponent = diskComponent;
         this.cleanupEmptyComponent = cleanupEmptyComponent;
     }
@@ -46,14 +52,18 @@
         bulkloaderChain.add(bulkloader);
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
+    @CriticalPath
     public void add(ITupleReference tuple) throws HyracksDataException {
         try {
             ITupleReference t = tuple;
-            for (IChainedComponentBulkLoader lsmBulkloader : bulkloaderChain) {
-                t = lsmBulkloader.add(t);
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                t = bulkloaderChain.get(i).add(t);
             }
-        } catch (Exception e) {
+        } catch (Throwable e) {
+            operation.setFailure(e);
             cleanupArtifacts();
             throw e;
         }
@@ -62,14 +72,18 @@
         }
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
+    @CriticalPath
     public void delete(ITupleReference tuple) throws HyracksDataException {
         try {
             ITupleReference t = tuple;
-            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
-                t = lsmOperation.delete(t);
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                t = bulkloaderChain.get(i).delete(t);
             }
-        } catch (Exception e) {
+        } catch (Throwable e) {
+            operation.setFailure(e);
             cleanupArtifacts();
             throw e;
         }
@@ -103,8 +117,14 @@
 
     @Override
     public void abort() throws HyracksDataException {
+        operation.setStatus(LSMIOOperationStatus.FAILURE);
         for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
             lsmOperation.abort();
         }
     }
+
+    @Override
+    public ILSMIOOperation getOperation() {
+        return operation;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 7bb24dc..dad3bdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -30,10 +31,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -242,22 +242,22 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
+    public ILSMIOOperation getIoOperation() {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
+    public void setIoOperation(ILSMIOOperation ioOperation) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
+    public void setParameters(Map<String, Object> map) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
+    public Map<String, Object> getParameters() {
         throw new UnsupportedOperationException();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 13911ef..c642d82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -49,10 +49,10 @@
             IIndexAccessParameters iap =
                     new IndexAccessParameters(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             ILSMIndexAccessor accessor = index.createAccessor(iap);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            accessor.scheduleFullMerge();
         } else if (immutableComponents.size() >= numComponents) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
         }
     }
 
@@ -108,7 +108,7 @@
                 throw new IllegalStateException();
             }
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
             return true;
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index e3ca9f1..4c2ddb6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -27,6 +27,8 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.common.IIndex;
 
@@ -84,7 +86,7 @@
 
     @Override
     public ILSMComponentId getId() {
-        return LSMComponentId.MISSING_COMPONENT_ID;
+        return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
     }
 
     @Override
@@ -144,9 +146,19 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
         return null;
     }
+
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public int getReaderCount() {
+        return 0;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
index 404c8e9..ab70ba1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -28,8 +28,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -39,15 +38,11 @@
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.util.trace.ITracer;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 public class ExternalIndexHarness extends LSMHarness {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled) {
-        super(lsmIndex, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
+    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled) {
+        super(lsmIndex, ioScheduler, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
     }
 
     @Override
@@ -104,14 +99,6 @@
             }
             ctx.setAccessingComponents(true);
         }
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
@@ -135,7 +122,7 @@
                             if (replicationEnabled) {
                                 componentsToBeReplicated.clear();
                                 componentsToBeReplicated.add((ILSMDiskComponent) c);
-                                lsmIndex.scheduleReplication(null, componentsToBeReplicated, false,
+                                lsmIndex.scheduleReplication(null, componentsToBeReplicated,
                                         ReplicationOperation.DELETE, opType);
                             }
                             ((ILSMDiskComponent) c).deactivateAndDestroy();
@@ -155,7 +142,7 @@
                             if (replicationEnabled) {
                                 componentsToBeReplicated.clear();
                                 componentsToBeReplicated.add(newComponent);
-                                triggerReplication(componentsToBeReplicated, false, opType);
+                                triggerReplication(componentsToBeReplicated, opType);
                             }
                             mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
                         }
@@ -206,54 +193,6 @@
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            // If the merge cannot be scheduled because there is already an ongoing merge on subset/all of the components, then
-            // whenever the current merge has finished, it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
-
-        ILSMDiskComponent newComponent = null;
-        try {
-            newComponent = lsmIndex.merge(operation);
-            ctx.setNewComponent(newComponent);
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            operation.getCallback().afterOperation(ctx);
-            newComponent.markAsValid(lsmIndex.isDurable());
-        } finally {
-            exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
-            operation.getCallback().afterFinalize(ctx);
-        }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
-        }
-    }
-
-    @Override
     public void addBulkLoadedComponent(ILSMDiskComponent c) throws HyracksDataException {
         c.markAsValid(lsmIndex.isDurable());
         synchronized (opTracker) {
@@ -261,7 +200,7 @@
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(c);
-                triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+                triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
             }
             // Enter the component
             enterComponent(c);
@@ -304,14 +243,13 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.afterFinalize(ctx);
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        return NoOpIoOperation.INSTANCE;
     }
 
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
     }
 
     @Override
@@ -354,7 +292,7 @@
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(diskComponent);
-                lsmIndex.scheduleReplication(null, componentsToBeReplicated, false, ReplicationOperation.DELETE, null);
+                lsmIndex.scheduleReplication(null, componentsToBeReplicated, ReplicationOperation.DELETE, null);
             }
             diskComponent.deactivateAndDestroy();
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
index d835021..b2a2e48 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
@@ -23,11 +23,10 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 
-public abstract class FlushOperation extends AbstractIoOperation implements Comparable<ILSMIOOperation> {
+public abstract class FlushOperation extends AbstractIoOperation {
 
     public FlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
             String indexIdentifier) {
@@ -35,9 +34,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.flush(this);
-        return true;
+        return getStatus();
     }
 
     @Override
@@ -65,14 +64,6 @@
     }
 
     @Override
-    public int compareTo(ILSMIOOperation o) {
-        if (o instanceof FlushOperation) {
-            return target.getFile().getName().compareTo(((FlushOperation) o).getTarget().getFile().getName());
-        }
-        return -1;
-    }
-
-    @Override
     public boolean equals(Object o) {
         return (o instanceof FlushOperation)
                 && Objects.equals(target.getFile().getName(), ((FlushOperation) o).target.getFile().getName());
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
new file mode 100644
index 0000000..354b1af
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public class IoOperationExecutor extends ThreadPoolExecutor {
+
+    private final ILSMIOOperationScheduler scheduler;
+    private final IIoOperationFailedCallback callback;
+    private final Map<String, ILSMIOOperation> runningFlushOperations;
+    private final Map<String, Throwable> failedGroups;
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations;
+
+    public IoOperationExecutor(ThreadFactory threadFactory, ILSMIOOperationScheduler scheduler,
+            IIoOperationFailedCallback callback, Map<String, ILSMIOOperation> runningFlushOperations,
+            Map<String, Deque<ILSMIOOperation>> waitingFlushOperations, Map<String, Throwable> failedGroups) {
+        super(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(), threadFactory);
+        this.scheduler = scheduler;
+        this.callback = callback;
+        this.runningFlushOperations = runningFlushOperations;
+        this.waitingFlushOperations = waitingFlushOperations;
+        this.failedGroups = failedGroups;
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
+        return new LSMIOOperationTask<>(callable);
+    }
+
+    @Override
+    protected void afterExecute(Runnable r, Throwable t) {
+        super.afterExecute(r, t);
+        LSMIOOperationTask<?> task = (LSMIOOperationTask<?>) r;
+        ILSMIOOperation executedOp = task.getOperation();
+        try {
+            doAfterExecute(executedOp, t);
+        } catch (Throwable th) { // NOSONAR must catch all
+            callback.schedulerFailed(scheduler, th);
+            shutdown();
+        }
+    }
+
+    private void doAfterExecute(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        final boolean failed = (t != null) || (executedOp.getStatus() == LSMIOOperationStatus.FAILURE);
+        if (failed) {
+            fail(executedOp, t != null ? t : executedOp.getFailure());
+        }
+        if (!failed || executedOp.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+            executedOp.complete(); // destroy if merge or successful flush
+        }
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            String id = executedOp.getIndexIdentifier();
+            synchronized (this) {
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    ILSMIOOperation op = waitingFlushOperations.get(id).poll();
+                    if (op != null) {
+                        scheduler.scheduleOperation(op);
+                    } else {
+                        waitingFlushOperations.remove(id);
+                    }
+                }
+            }
+        }
+    }
+
+    private void fail(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        callback.operationFailed(executedOp, t);
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            executedOp.complete();
+            // Doesn't make sense to process further flush requests... Mark the operation group permanently failed
+            // Fail other scheduled operations
+            synchronized (this) {
+                String id = executedOp.getIndexIdentifier();
+                failedGroups.put(id, t);
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    Deque<ILSMIOOperation> ops = waitingFlushOperations.remove(id);
+                    ILSMIOOperation next = ops.poll();
+                    while (next != null) {
+                        next.setFailure(new RuntimeException("Operation group " + id + " has permanently failed", t));
+                        next.setStatus(LSMIOOperationStatus.FAILURE);
+                        next.complete();
+                        next = ops.poll();
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
index 442af56..c7990a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
@@ -25,11 +25,11 @@
 
     public static final long NOT_FOUND = -1;
 
-    // Use to handle legacy datasets which do not have the component Id
-    public static final ILSMComponentId MISSING_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
+    // Used to represent an empty index with no components
+    public static final LSMComponentId EMPTY_INDEX_LAST_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
 
     // A default component id used for bulk loaded component
-    public static final ILSMComponentId DEFAULT_COMPONENT_ID = new LSMComponentId(0, 0);
+    public static final LSMComponentId DEFAULT_COMPONENT_ID = new LSMComponentId(0, 0);
 
     private long minId;
 
@@ -46,10 +46,12 @@
         this.maxId = maxId;
     }
 
+    @Override
     public long getMinId() {
         return this.minId;
     }
 
+    @Override
     public long getMaxId() {
         return this.maxId;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
index e174153..3da57fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
@@ -28,18 +28,22 @@
  */
 public class LSMComponentIdGenerator implements ILSMComponentIdGenerator {
 
+    private final int numComponents;
+    private int currentComponentIndex;
     protected long previousTimestamp = -1L;
-
     private ILSMComponentId componentId;
 
-    public LSMComponentIdGenerator() {
+    public LSMComponentIdGenerator(int numComponents) {
+        this.numComponents = numComponents;
         refresh();
+        currentComponentIndex = 0;
     }
 
     @Override
     public void refresh() {
         long ts = getCurrentTimestamp();
         componentId = new LSMComponentId(ts, ts);
+        currentComponentIndex = (currentComponentIndex + 1) % numComponents;
     }
 
     @Override
@@ -47,6 +51,11 @@
         return componentId;
     }
 
+    @Override
+    public int getCurrentComponentIndex() {
+        return currentComponentIndex;
+    }
+
     protected long getCurrentTimestamp() {
         long timestamp = System.currentTimeMillis();
         while (timestamp <= previousTimestamp) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
deleted file mode 100644
index d288ec8..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.common.IResource;
-
-/**
- * A default implementation of {@link ILSMComponentIdGeneratorFactory}.
- *
- */
-public class LSMComponentIdGeneratorFactory implements ILSMComponentIdGeneratorFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-        return new LSMComponentIdGenerator();
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 644508e..4d840b0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -42,8 +42,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -51,9 +52,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.util.IOOperationUtils;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.util.annotations.CriticalPath;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.ITracer.Scope;
 import org.apache.logging.log4j.Level;
@@ -64,6 +65,7 @@
     private static final Logger LOGGER = LogManager.getLogger();
 
     protected final ILSMIndex lsmIndex;
+    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ComponentReplacementContext componentReplacementCtx;
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMOperationTracker opTracker;
@@ -73,9 +75,10 @@
     protected ITracer tracer;
     protected long traceCategory;
 
-    public LSMHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled, ITracer tracer) {
+    public LSMHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled, ITracer tracer) {
         this.lsmIndex = lsmIndex;
+        this.ioScheduler = ioScheduler;
         this.opTracker = opTracker;
         this.mergePolicy = mergePolicy;
         this.tracer = tracer;
@@ -100,58 +103,12 @@
             synchronized (opTracker) {
                 while (true) {
                     lsmIndex.getOperationalComponents(ctx);
-                    // Before entering the components, prune those corner cases that indeed should not proceed.
-                    switch (opType) {
-                        case FLUSH:
-                            // if the lsm index does not have memory components allocated, then nothing to flush
-                            if (!lsmIndex.isMemoryComponentsAllocated()) {
-                                return false;
-                            }
-                            ILSMMemoryComponent flushingComponent =
-                                    (ILSMMemoryComponent) ctx.getComponentHolder().get(0);
-                            if (!flushingComponent.isModified()) {
-                                recycle(flushingComponent);
-                                return false;
-                            }
-                            if (flushingComponent.getWriterCount() > 0) {
-                                /*
-                                 * This case is a case where even though FLUSH log was flushed to disk
-                                 * and scheduleFlush is triggered, the current in-memory component (whose state was
-                                 * changed to READABLE_WRITABLE (RW) from READABLE_UNWRITABLE(RU) before FLUSH log
-                                 * was written to log tail (which is memory buffer of log file) and then the state was
-                                 * changed back to RW (as shown in the following scenario)) can have writers based on
-                                 * the current code base/design. Thus, the writer count of the component may be greater
-                                 * than 0. if this happens, intead of throwing exception, scheduleFlush() deal with
-                                 * this situation by not flushing the component.
-                                 * for more detailed information: ASTERIXDB-1027
-                                 */
-                                return false;
-                            }
-                            break;
-                        case MERGE:
-                            if (ctx.getComponentHolder().size() < 2
-                                    && ctx.getOperation() != IndexOperation.DELETE_DISK_COMPONENTS) {
-                                // There is only a single component. There is nothing to merge.
-                                return false;
-                            }
-                            break;
-                        default:
-                            break;
-                    }
                     if (enterComponents(ctx, opType)) {
                         return true;
                     } else if (isTryOperation) {
                         return false;
                     }
                     try {
-                        // Flush and merge operations should never reach this wait call,
-                        // because they are always try operations. If they fail to enter the components,
-                        // then it means that there are an ongoing flush/merge operation on
-                        // the same components, so they should not proceed.
-                        if (opType == LSMOperationType.MODIFICATION) {
-                            // before waiting, make sure the index is in a modifiable state to avoid waiting forever.
-                            ensureIndexModifiable();
-                        }
                         opTracker.wait();
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
@@ -166,20 +123,7 @@
         }
     }
 
-    private void recycle(ILSMMemoryComponent flushingComponent) throws HyracksDataException {
-        if (flushingComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
-            //The mutable component has not been modified by any writer.
-            // There is nothing to flush. Since the component is empty, set its state back
-            // to READABLE_WRITABLE only when it's state has been set to READABLE_UNWRITABLE
-            flushingComponent.setState(ComponentState.READABLE_WRITABLE);
-            opTracker.notifyAll(); // NOSONAR: Always synchronized from caller
-            // Call recycled only when we change it's state is reset back to READABLE_WRITABLE
-            // Otherwise, if the component is in other state, e.g., INACTIVE, or
-            // READABLE_UNWRITABLE_FLUSHING, it's not considered as being recycled here.
-            lsmIndex.getIOOperationCallback().recycled(flushingComponent, false);
-        }
-    }
-
+    @CriticalPath
     protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType)
             throws HyracksDataException {
         validateOperationEnterComponentsState(ctx);
@@ -187,9 +131,11 @@
         int numEntered = 0;
         boolean entranceSuccessful = false;
         try {
-            for (ILSMComponent c : components) {
-                boolean isMutableComponent = numEntered == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
-                if (!c.threadEnter(opType, isMutableComponent)) {
+            final int componentsCount = components.size();
+            for (int i = 0; i < componentsCount; i++) {
+                final ILSMComponent component = components.get(i);
+                boolean isMutableComponent = numEntered == 0 && component.getType() == LSMComponentType.MEMORY;
+                if (!component.threadEnter(opType, isMutableComponent)) {
                     break;
                 }
                 numEntered++;
@@ -202,14 +148,14 @@
             throw e;
         } finally {
             if (!entranceSuccessful) {
-                int i = 0;
-                for (ILSMComponent c : components) {
+                final int componentsCount = components.size();
+                for (int i = 0; i < componentsCount; i++) {
+                    final ILSMComponent component = components.get(i);
                     if (numEntered == 0) {
                         break;
                     }
-                    boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
-                    c.threadExit(opType, true, isMutableComponent);
-                    i++;
+                    boolean isMutableComponent = i == 0 && component.getType() == LSMComponentType.MEMORY;
+                    component.threadExit(opType, true, isMutableComponent);
                     numEntered--;
                 }
             }
@@ -218,44 +164,26 @@
             return false;
         }
         ctx.setAccessingComponents(true);
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case FLUSH:
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                // Changing the flush status should *always* precede changing the mutable component.
-                lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
-                lsmIndex.changeMutableComponent();
-                // Notify all waiting threads whenever a flush has been scheduled since they will check
-                // again if they can grab and enter the mutable component.
-                opTracker.notifyAll(); // NOSONAR: Always called from a synchronized block
-                break;
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                break;
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
 
+    @CriticalPath
     private void doExitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
             ILSMDiskComponent newComponent, boolean failedOperation) throws HyracksDataException {
-        /**
+        /*
          * FLUSH and MERGE operations should always exit the components
          * to notify waiting threads.
          */
         if (!ctx.isAccessingComponents() && opType != LSMOperationType.FLUSH && opType != LSMOperationType.MERGE) {
             return;
         }
-        List<ILSMDiskComponent> inactiveDiskComponents = null;
+        List<ILSMDiskComponent> inactiveDiskComponents;
         List<ILSMDiskComponent> inactiveDiskComponentsToBeDeleted = null;
         try {
             synchronized (opTracker) {
                 try {
-                    /**
+                    /*
                      * [flow control]
                      * If merge operations are lagged according to the merge policy,
                      * flushing in-memory components are hold until the merge operation catches up.
@@ -319,7 +247,7 @@
                 try {
                     //schedule a replication job to delete these inactive disk components from replicas
                     if (replicationEnabled) {
-                        lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted, false,
+                        lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted,
                                 ReplicationOperation.DELETE, opType);
                     }
                     for (ILSMDiskComponent c : inactiveDiskComponentsToBeDeleted) {
@@ -343,10 +271,12 @@
                 // newComponent is null if the flush op. was not performed.
                 if (!failedOperation && newComponent != null) {
                     lsmIndex.addDiskComponent(newComponent);
+                    // TODO: The following should also replicate component Id
+                    // even if empty component
                     if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
                         componentsToBeReplicated.clear();
                         componentsToBeReplicated.add(newComponent);
-                        triggerReplication(componentsToBeReplicated, false, opType);
+                        triggerReplication(componentsToBeReplicated, opType);
                     }
                     mergePolicy.diskComponentAdded(lsmIndex, false);
                 }
@@ -358,7 +288,7 @@
                     if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
                         componentsToBeReplicated.clear();
                         componentsToBeReplicated.add(newComponent);
-                        triggerReplication(componentsToBeReplicated, false, opType);
+                        triggerReplication(componentsToBeReplicated, opType);
                     }
                     mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
                 }
@@ -368,13 +298,16 @@
         }
     }
 
+    @CriticalPath
     private void exitOperationalComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
             boolean failedOperation) throws HyracksDataException {
         // First check if there is any action that is needed to be taken
         // based on the state of each component.
-        for (int i = 0; i < ctx.getComponentHolder().size(); i++) {
-            ILSMComponent c = ctx.getComponentHolder().get(i);
-            boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+        final List<ILSMComponent> componentHolder = ctx.getComponentHolder();
+        final int componentsCount = componentHolder.size();
+        for (int i = 0; i < componentsCount; i++) {
+            final ILSMComponent c = componentHolder.get(i);
+            boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY;
             c.threadExit(opType, failedOperation, isMutableComponent);
             if (c.getType() == LSMComponentType.MEMORY) {
                 switch (c.getState()) {
@@ -386,7 +319,6 @@
                         break;
                     case INACTIVE:
                         tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex.toString());
-                        ((AbstractLSMMemoryComponent) c).reset();
                         // Notify all waiting threads whenever the mutable component's state
                         // has changed to inactive. This is important because even though we switched
                         // the mutable components, it is possible that the component that we just
@@ -548,129 +480,137 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.afterFinalize(ctx);
-            return;
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation flush;
+        LOGGER.info("Flush is being scheduled on {}", lsmIndex);
+        if (!lsmIndex.isMemoryComponentsAllocated()) {
+            lsmIndex.allocateMemoryComponents();
         }
-        lsmIndex.scheduleFlush(ctx, callback);
+        synchronized (opTracker) {
+            try {
+                flush = lsmIndex.createFlushOperation(ctx);
+            } finally {
+                // Notify all waiting threads whenever a flush has been scheduled since they will check
+                // again if they can grab and enter the mutable component.
+                opTracker.notifyAll();
+            }
+        }
+        ioScheduler.scheduleOperation(flush);
+        return flush;
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a flush operation for index: " + lsmIndex + " ...");
+            LOGGER.info("Started a flush operation for index: {}", lsmIndex);
+        }
+        synchronized (opTracker) {
+            while (!enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH)) {
+                try {
+                    opTracker.wait();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
         }
         try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.flush(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Flush failed on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.FLUSH, newComponent, failedOperation);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterFinalize(ctx);
-
-            }
+            doIo(operation);
         } finally {
-            /*
-             * Completion of flush/merge operations is done explicitly here to make sure all generated files during
-             * io operations is completed before the io operation is declared complete
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
         }
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the flush operation for index: " + lsmIndex);
+            LOGGER.info("Finished the flush operation for index: {}. Result: ", lsmIndex, operation.getStatus());
+        }
+    }
+
+    public void doIo(ILSMIOOperation operation) {
+        try {
+            operation.getCallback().beforeOperation(operation);
+            ILSMDiskComponent newComponent = operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+                    ? lsmIndex.flush(operation) : lsmIndex.merge(operation);
+            operation.setNewComponent(newComponent);
+            operation.getCallback().afterOperation(operation);
+            if (newComponent != null) {
+                newComponent.markAsValid(lsmIndex.isDurable());
+            }
+        } catch (Throwable e) { // NOSONAR Must catch all
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(e);
+            if (LOGGER.isErrorEnabled()) {
+                LOGGER.log(Level.ERROR, "{} operation failed on {}", operation.getIOOpertionType(), lsmIndex, e);
+            }
+        } finally {
+            try {
+                operation.getCallback().afterFinalize(operation);
+            } catch (Throwable th) {// NOSONAR Must catch all
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(th);
+                if (LOGGER.isErrorEnabled()) {
+                    LOGGER.log(Level.ERROR, "{} operation.afterFinalize failed on {}", operation.getIOOpertionType(),
+                            lsmIndex, th);
+                }
+            }
+        }
+        // if the operation failed, we need to cleanup files
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            lsmIndex.cleanUpFilesForFailedOperation(operation);
         }
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            LOGGER.info("Failed to enter components for merge operation. Calling finalize");
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.info("Started a merge operation for index: {}", lsmIndex);
         }
-        lsmIndex.scheduleMerge(ctx, callback);
+        synchronized (opTracker) {
+            enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE);
+        }
+        try {
+            doIo(operation);
+        } finally {
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
+        }
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.info("Finished the merge operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
+        }
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+    public ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            operation = lsmIndex.createMergeOperation(ctx);
+        }
+        ioScheduler.scheduleOperation(operation);
+        return operation;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            fullMergeIsRequested.set(true);
+            ctx.getComponentsToBeMerged().addAll(lsmIndex.getDiskComponents());
+            operation = lsmIndex.createMergeOperation(ctx);
+            if (operation != NoOpIoOperation.INSTANCE) {
+                fullMergeIsRequested.set(false);
+            }
             // If the merge cannot be scheduled because there is already an ongoing merge on
             // subset/all of the components, then whenever the current merge has finished,
             // it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
         }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
-        try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.merge(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR: Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Failed merge operation on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.MERGE, newComponent, failedOperation);
-                operation.getCallback().afterFinalize(ctx);
-            }
-        } finally {
-            /*
-             * Completion of the merge operation is called here to and not on afterOperation because
-             * deletion of old components comes after afterOperation is called and the number of
-             * io operation should not be decremented before the operation is complete to avoid
-             * index destroy from competing with the merge on deletion of the files.
-             * The order becomes:
-             * 1. scheduleMerge
-             * 2. enterComponents
-             * 3. beforeOperation (increment the numOfIoOperations)
-             * 4. merge
-             * 5. exitComponents
-             * 6. afterOperation (no op)
-             * 7. delete components
-             * 8. completeOperation (decrement the numOfIoOperations)
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
-        }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
-        }
+        ioScheduler.scheduleOperation(operation);
+        return operation;
     }
 
     @Override
@@ -681,7 +621,7 @@
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(c);
-                triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+                triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
             }
             mergePolicy.diskComponentAdded(lsmIndex, false);
         }
@@ -692,20 +632,20 @@
         return opTracker;
     }
 
-    protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleReplication(lsmComponents, bulkload, opType);
+        accessor.scheduleReplication(lsmComponents, opType);
     }
 
     @Override
     public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
-            boolean bulkload, LSMOperationType opType) throws HyracksDataException {
+            LSMOperationType opType) throws HyracksDataException {
         //enter the LSM components to be replicated to prevent them from being deleted until they are replicated
         if (!getAndEnterComponents(ctx, LSMOperationType.REPLICATE, false)) {
             return;
         }
-        lsmIndex.scheduleReplication(ctx, lsmComponents, bulkload, ReplicationOperation.REPLICATE, opType);
+        lsmIndex.scheduleReplication(ctx, lsmComponents, ReplicationOperation.REPLICATE, opType);
     }
 
     @Override
@@ -771,36 +711,6 @@
         }
     }
 
-    /***
-     * Ensures the index is in a modifiable state (no failed flushes)
-     *
-     * @throws HyracksDataException
-     *             if the index is not in a modifiable state
-     */
-    private void ensureIndexModifiable() throws HyracksDataException {
-        // if current memory component has a flush request, it means that flush didn't start for it
-        if (lsmIndex.hasFlushRequestForCurrentMutableComponent()) {
-            return;
-        }
-        // find if there is any memory component which is in a writable state or eventually will be in a writable state
-        for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
-            switch (memoryComponent.getState()) {
-                case INACTIVE:
-                    // will be activated on next modification
-                case UNREADABLE_UNWRITABLE:
-                    // flush completed successfully but readers are still inside
-                case READABLE_WRITABLE:
-                    // writable
-                case READABLE_UNWRITABLE_FLUSHING:
-                    // flush is ongoing
-                    return;
-                default:
-                    // continue to the next component
-            }
-        }
-        throw HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-    }
-
     /**
      * Waits for any lagging merge operations to finish to avoid breaking
      * the merge policy (i.e. adding a new disk component can make the
@@ -824,51 +734,52 @@
         }
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
     public void deleteComponents(ILSMIndexOperationContext ctx, Predicate<ILSMComponent> predicate)
             throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper ioCallback =
-                new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
-        boolean deleteMemoryComponent = false;
+        ILSMIOOperation ioOperation = null;
+        // We need to always start the component delete from current memory component.
+        // This will ensure Primary and secondary component id still matches after component delete
+        if (!lsmIndex.isMemoryComponentsAllocated()) {
+            lsmIndex.allocateMemoryComponents();
+        }
         synchronized (opTracker) {
             waitForFlushesAndMerges();
-            ensureNoFailedFlush();
-            if (lsmIndex.isMemoryComponentsAllocated()) {
-                // We always start with the memory component
-                ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
-                deleteMemoryComponent = predicate.test(memComponent);
-                if (deleteMemoryComponent) {
-                    // schedule a delete for flushed component
-                    ctx.reset();
-                    ctx.setOperation(IndexOperation.DELETE_MEMORY_COMPONENT);
-                    // ScheduleFlush is actually a try operation
-                    scheduleFlush(ctx, ioCallback);
-                } else {
-                    // shouldn't try to delete disk components while memory component is still there
-                    return;
-                }
+            // We always start with the memory component
+            ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+            if (predicate.test(memComponent)) {
+                // schedule a delete for flushed component
+                ctx.reset();
+                ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
+                ioOperation = scheduleFlush(ctx);
+            } else {
+                // since we're not deleting the memory component, we can't delete any previous component
+                return;
             }
         }
         // Here, we are releasing the opTracker to allow other operations:
         // (searches, delete flush we will schedule, delete merge we will schedule).
-        if (deleteMemoryComponent) {
-            IOOperationUtils.waitForIoOperation(ioCallback);
+        try {
+            ioOperation.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(ioOperation.getFailure());
         }
         ctx.reset();
-        ioCallback = new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
-        ctx.setOperation(IndexOperation.DELETE_DISK_COMPONENTS);
+        ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
         List<ILSMDiskComponent> toBeDeleted;
         synchronized (opTracker) {
             waitForFlushesAndMerges();
-            // Ensure that current memory component is empty and that no failed flushes happened so far
-            // This is a workaround until ASTERIXDB-2106 is fixed
-            ensureNoFailedFlush();
             List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
             for (ILSMDiskComponent component : diskComponents) {
                 if (predicate.test(component)) {
                     ctx.getComponentsToBeMerged().add(component);
                 } else {
-                    // can't delete components while newer ones are still there
+                    // Can't delete older components when newer one is still there
                     break;
                 }
             }
@@ -877,9 +788,17 @@
             }
             toBeDeleted = new ArrayList<>(ctx.getComponentsToBeMerged());
             // ScheduleMerge is actually a try operation
-            scheduleMerge(ctx, ioCallback);
+            ioOperation = scheduleMerge(ctx);
         }
-        IOOperationUtils.waitForIoOperation(ioCallback);
+        try {
+            ioOperation.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(ioOperation.getFailure());
+        }
         synchronized (opTracker) {
             // ensure that merge has succeeded
             for (ILSMDiskComponent component : toBeDeleted) {
@@ -890,21 +809,6 @@
         }
     }
 
-    /**
-     * This can only be called in the steady state where:
-     * 1. no scheduled flushes
-     * 2. no incoming data
-     *
-     * @throws HyracksDataException
-     */
-    private void ensureNoFailedFlush() throws HyracksDataException {
-        for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
-            if (memoryComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
-                throw HyracksDataException.create(ErrorCode.A_FLUSH_OPERATION_HAS_FAILED);
-            }
-        }
-    }
-
     private void waitForFlushesAndMerges() throws HyracksDataException {
         while (flushingOrMerging()) {
             try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
index 5e105a4..10074f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -22,62 +22,81 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 
 public class LSMIndexDiskComponentBulkLoader implements IIndexBulkLoader {
     private final AbstractLSMIndex lsmIndex;
     private final ILSMDiskComponentBulkLoader componentBulkLoader;
-    private ILSMIndexOperationContext opCtx;
+    private final ILSMIndexOperationContext opCtx;
 
     public LSMIndexDiskComponentBulkLoader(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext opCtx, float fillFactor,
             boolean verifyInput, long numElementsHint) throws HyracksDataException {
         this.lsmIndex = lsmIndex;
         this.opCtx = opCtx;
-        // Note that by using a flush target file name, we state that the
-        // new bulk loaded component is "newer" than any other merged component.
-        opCtx.setNewComponent(lsmIndex.createBulkLoadTarget());
-        this.componentBulkLoader =
-                opCtx.getNewComponent().createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+        this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
+                fillFactor, verifyInput, numElementsHint, false, true, true);
     }
 
     public ILSMDiskComponent getComponent() {
-        return opCtx.getNewComponent();
+        return opCtx.getIoOperation().getNewComponent();
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
     public void add(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.add(tuple);
+        try {
+            componentBulkLoader.add(tuple);
+        } catch (Throwable th) {
+            opCtx.getIoOperation().setFailure(th);
+            throw th;
+        }
     }
 
+    @SuppressWarnings("squid:S1181")
     public void delete(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.delete(tuple);
+        try {
+            componentBulkLoader.delete(tuple);
+        } catch (Throwable th) {
+            opCtx.getIoOperation().setFailure(th);
+            throw th;
+        }
     }
 
     @Override
     public void end() throws HyracksDataException {
         try {
-            componentBulkLoader.end();
-            if (opCtx.getNewComponent().getComponentSize() > 0) {
-                //TODO(amoudi): Ensure Bulk load follow the same lifecycle Other Operations (Flush, Merge, etc).
-                //then after operation should be called from harness as well
-                //https://issues.apache.org/jira/browse/ASTERIXDB-1764
-                lsmIndex.getIOOperationCallback().afterOperation(opCtx);
-                lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getNewComponent());
+            try {
+                lsmIndex.getIOOperationCallback().afterOperation(opCtx.getIoOperation());
+                componentBulkLoader.end();
+            } catch (Throwable th) { // NOSONAR Must not call afterFinalize without setting failure
+                opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
+                opCtx.getIoOperation().setFailure(th);
+                throw th;
+            } finally {
+                lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+            }
+            if (opCtx.getIoOperation().getStatus() == LSMIOOperationStatus.SUCCESS
+                    && opCtx.getIoOperation().getNewComponent().getComponentSize() > 0) {
+                lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getIoOperation().getNewComponent());
             }
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
     @Override
     public void abort() throws HyracksDataException {
+        opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
         try {
-            componentBulkLoader.abort();
-            opCtx.setNewComponent(null);
-            lsmIndex.getIOOperationCallback().afterOperation(opCtx);
+            try {
+                componentBulkLoader.abort();
+            } finally {
+                lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+            }
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 445a005..12caec4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -125,7 +125,6 @@
                     rangeCursors[i].close();
                 }
             }
-            rangeCursors = null;
         } finally {
             if (lsmHarness != null) {
                 lsmHarness.endSearch(opCtx);
@@ -247,9 +246,9 @@
         }
     }
 
-    public class PriorityQueueElement {
+    public static class PriorityQueueElement {
         private ITupleReference tuple;
-        private int cursorIndex;
+        private final int cursorIndex;
 
         public PriorityQueueElement(int cursorIndex) {
             tuple = null;
@@ -269,7 +268,7 @@
         }
     }
 
-    public class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
+    public static class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
 
         protected MultiComparator cmp;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 1548f86..8412b8c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -120,12 +119,12 @@
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
@@ -135,33 +134,32 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
-    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ctx.setOperation(IndexOperation.REPLICATE);
         ctx.getComponentsToBeReplicated().clear();
         ctx.getComponentsToBeReplicated().addAll(lsmComponents);
-        lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+        lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
new file mode 100644
index 0000000..21c52d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+
+public class LoadOperation extends AbstractIoOperation {
+
+    private final LSMComponentFileReferences fileReferences;
+    private final Map<String, Object> parameters;
+
+    public LoadOperation(LSMComponentFileReferences fileReferences, ILSMIOOperationCallback callback,
+            String indexIdentifier, Map<String, Object> parameters) {
+        super(null, fileReferences.getInsertIndexFileReference(), callback, indexIdentifier);
+        this.fileReferences = fileReferences;
+        this.parameters = parameters;
+    }
+
+    @Override
+    public final LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.LOAD;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return fileReferences;
+    }
+
+    @Override
+    public void sync() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return parameters;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
index ec2305d..9d7c449 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
@@ -41,9 +41,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.merge(this);
-        return true;
+        return getStatus();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index 9b25471..9bd5007 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -220,4 +220,8 @@
         vbc.deleteFile(file);
     }
 
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
index 3432624..600a02d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
@@ -20,15 +20,20 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.IResource;
 
-public enum NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-    INSTANCE;
+public class NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final NoOpIOOperationCallbackFactory INSTANCE = new NoOpIOOperationCallbackFactory();
+
+    private NoOpIOOperationCallbackFactory() {
+    }
 
     @Override
     public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) {
@@ -47,22 +52,7 @@
         }
 
         @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
+        public void recycled(ILSMMemoryComponent component) {
             // Do nothing.
         }
 
@@ -70,5 +60,35 @@
         public void allocated(ILSMMemoryComponent component) {
             // Do nothing.
         }
+
+        @Override
+        public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void completed(ILSMIOOperation operation) {
+            // Do nothing.
+        }
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return 0;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
new file mode 100644
index 0000000..f57c4ef
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
+
+public class NoOpIoOperation implements ILSMIOOperation {
+    public static final NoOpIoOperation INSTANCE = new NoOpIoOperation();
+
+    private NoOpIoOperation() {
+    }
+
+    @Override
+    public IODeviceHandle getDevice() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getIndexIdentifier() {
+        return NoOpIoOperation.class.getSimpleName();
+    }
+
+    @Override
+    public LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.NOOP;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public FileReference getTarget() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIndexAccessor getAccessor() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        // No Op
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        // No Op
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        // No Op
+    }
+
+    @Override
+    public void complete() {
+        // No Op
+    }
+
+    @Override
+    public void sync() {
+        // No Op
+    }
+
+    @Override
+    public void addCompleteListener(IoOperationCompleteListener listener) {
+        listener.completed(this);
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return null;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
new file mode 100644
index 0000000..3b2b2c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class NoOpIoOperationFailedCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    public static final NoOpIoOperationFailedCallback INSTANCE = new NoOpIoOperationFailedCallback();
+
+    private NoOpIoOperationFailedCallback() {
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} failed", operation, t);
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler failed", failure);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
index 73b5d20..b571e58 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 
 public class PrefixMergePolicy implements ILSMMergePolicy {
@@ -59,8 +58,7 @@
         }
 
         if (fullMergeIsRequested) {
-            ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFullMerge();
             return;
         }
 
@@ -244,8 +242,7 @@
 
         // Reverse the components order back to its original order
         Collections.reverse(mergableComponents);
-        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleMerge(index.getIOOperationCallback(), mergableComponents);
+        index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleMerge(mergableComponents);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
deleted file mode 100644
index 2c16be0..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-/**
- * This class is for testing. It's basically a way to get the new/old component info from the
- * harness callback simply.
- */
-
-public class StubIOOperationCallback implements ILSMIOOperationCallback {
-
-    private ILSMIndexOperationContext opCtx = null;
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Not interested in this
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        this.opCtx = opCtx;
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Redundant info from after
-    }
-
-    public List<ILSMDiskComponent> getLastOldComponents() {
-        return opCtx.getComponentsToBeMerged();
-    }
-
-    public ILSMDiskComponent getLastNewComponent() {
-        return opCtx.getNewComponent();
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
-        // Not interested in this
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) {
-        // Not interested in this
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
index 506dcea..ae3b4e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -18,8 +18,13 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -27,18 +32,76 @@
 
 public class SynchronousScheduler implements ILSMIOOperationScheduler {
     private static final Logger LOGGER = LogManager.getLogger();
-    public static final SynchronousScheduler INSTANCE = new SynchronousScheduler();
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new ConcurrentHashMap<>();
+    private final Map<String, Throwable> failedGroups = new ConcurrentHashMap<>();
+    private final IIoOperationFailedCallback failureCallback;
 
-    private SynchronousScheduler() {
+    public SynchronousScheduler(IIoOperationFailedCallback failureCallback) {
+        this.failureCallback = failureCallback;
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+    public void scheduleOperation(ILSMIOOperation operation) {
+        try {
+            before(operation);
+            if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                return;
+            }
+            run(operation);
+        } catch (Throwable e) { // NOSONAR: Must catch them all
+            throw new IllegalStateException(e);
+        } finally {
+            after(operation);
+        }
+    }
+
+    private void run(ILSMIOOperation operation) {
         try {
             operation.call();
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "IO Operation failed", e);
-            throw HyracksDataException.create(e);
+        } catch (Throwable th) { // NOSONAR Must catch all
+            LOGGER.log(Level.ERROR, "IO Operation failed", th);
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(th);
+        }
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            failureCallback.operationFailed(operation, operation.getFailure());
+        }
+    }
+
+    private void after(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                runningFlushOperations.remove(operation.getIndexIdentifier());
+                if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    failedGroups.putIfAbsent(operation.getIndexIdentifier(), operation.getFailure());
+                }
+                operation.complete();
+                runningFlushOperations.notifyAll();
+            }
+        } else {
+            operation.complete();
+        }
+    }
+
+    private void before(ILSMIOOperation operation) throws InterruptedException {
+        String id = operation.getIndexIdentifier();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                while (true) {
+                    if (failedGroups.containsKey(id)) {
+                        operation.setStatus(LSMIOOperationStatus.FAILURE);
+                        operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                                failedGroups.get(id)));
+                        return;
+                    }
+                    if (runningFlushOperations.containsKey(id)) {
+                        runningFlushOperations.wait();
+                    } else {
+                        runningFlushOperations.put(id, operation);
+                        break;
+                    }
+                }
+            }
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
index 823db04..132d5cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -22,12 +22,16 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 
-public enum SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
-    INSTANCE;
+public class SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
+    private static final long serialVersionUID = 1L;
+    public static final SynchronousSchedulerProvider INSTANCE = new SynchronousSchedulerProvider();
+
+    private SynchronousSchedulerProvider() {
+    }
 
     @Override
     public ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx) {
-        return SynchronousScheduler.INSTANCE;
+        return new SynchronousScheduler(NoOpIoOperationFailedCallback.INSTANCE);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index a7fbe06..07eeb28 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -58,7 +58,7 @@
         if (opType == LSMOperationType.MODIFICATION && threadRefCount.decrementAndGet() == 0
                 && index.hasFlushRequestForCurrentMutableComponent()) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFlush(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback(index));
+            accessor.scheduleFlush();
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 7238f8e..f1172f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -19,12 +19,16 @@
 
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.util.Map;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.ITracer.Scope;
 import org.apache.hyracks.util.trace.TraceUtils;
@@ -53,10 +57,7 @@
         if (tracer.isEnabled(traceCategory)) {
             tracer.instant("schedule-" + ioOpName, traceCategory, Scope.p,
                     "{\"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
-        }
-        if (tracer.isEnabled(traceCategory)) {
-            return ioOp instanceof Comparable ? new ComparableTracedIOOperation(ioOp, tracer, traceCategory)
-                    : new TracedIOOperation(ioOp, tracer, traceCategory);
+            return new TracedIOOperation(ioOp, tracer, traceCategory);
         }
         return ioOp;
     }
@@ -86,7 +87,7 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         final String name = getTarget().getRelativePath();
         final long tid = tracer.durationB(name, traceCategory, null);
         try {
@@ -111,33 +112,54 @@
     public LSMComponentFileReferences getComponentFiles() {
         return ioOp.getComponentFiles();
     }
+
+    @Override
+    public Throwable getFailure() {
+        return ioOp.getFailure();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        ioOp.setFailure(failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return ioOp.getStatus();
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        ioOp.setStatus(status);
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return ioOp.getNewComponent();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        ioOp.setNewComponent(component);
+    }
+
+    @Override
+    public void complete() {
+        ioOp.complete();
+    }
+
+    @Override
+    public void sync() throws InterruptedException {
+        ioOp.sync();
+    }
+
+    @Override
+    public void addCompleteListener(IoOperationCompleteListener listener) {
+        ioOp.addCompleteListener(listener);
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return ioOp.getParameters();
+    }
 }
-
-class ComparableTracedIOOperation extends TracedIOOperation implements Comparable<ILSMIOOperation> {
-
-    protected ComparableTracedIOOperation(ILSMIOOperation ioOp, ITracer trace, long traceCategory) {
-        super(ioOp, trace, traceCategory);
-    }
-
-    @Override
-    public int hashCode() {
-        return this.ioOp.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        return other instanceof ILSMIOOperation && compareTo((ILSMIOOperation) other) == 0;
-    }
-
-    @Override
-    public int compareTo(ILSMIOOperation other) {
-        final ILSMIOOperation myIoOp = this.ioOp;
-        if (myIoOp instanceof Comparable && other instanceof ComparableTracedIOOperation) {
-            return ((Comparable) myIoOp).compareTo(((ComparableTracedIOOperation) other).getIoOp());
-        }
-        LOGGER.warn("Comparing ioOps of type " + myIoOp.getClass().getSimpleName() + " and "
-                + other.getClass().getSimpleName() + " in " + getClass().getSimpleName());
-        return Integer.signum(hashCode() - other.hashCode());
-    }
-
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index ef9852d..e63a5b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -462,4 +462,10 @@
         map.put("open", open);
         return map;
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
deleted file mode 100644
index 9bd873d..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.util;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class IOOperationUtils {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private IOOperationUtils() {
-    }
-
-    public static void waitForIoOperation(BlockingIOOperationCallbackWrapper ioCallback) throws HyracksDataException {
-        // Note that the following call assumes that the io operation has succeeded.
-        try {
-            ioCallback.waitForIO();
-        } catch (InterruptedException e) {
-            LOGGER.log(Level.WARN, "Operation has been interrupted. returning");
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
index 6d4b0a7..1779527 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
@@ -25,8 +25,11 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class LSMComponentIdUtils {
+    private static final Logger LOGGER = LogManager.getLogger();
 
     private static final MutableArrayValueReference COMPONENT_ID_MIN_KEY =
             new MutableArrayValueReference("Component_Id_Min".getBytes());
@@ -43,7 +46,9 @@
         long minId = ComponentUtils.getLong(metadata, COMPONENT_ID_MIN_KEY, LSMComponentId.NOT_FOUND, buffer);
         long maxId = ComponentUtils.getLong(metadata, COMPONENT_ID_MAX_KEY, LSMComponentId.NOT_FOUND, buffer);
         if (minId == LSMComponentId.NOT_FOUND || maxId == LSMComponentId.NOT_FOUND) {
-            return LSMComponentId.MISSING_COMPONENT_ID;
+            LOGGER.warn("Invalid component id {} was persisted to a component metadata",
+                    LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
+            return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
         } else {
             return new LSMComponentId(minId, maxId);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 4eb7728..f2fb9af 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -95,14 +95,14 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory);
+                    metadataPageManagerFactory, serviceCtx.getTracer());
         } else {
             return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
                     tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache, file.getAbsolutePath(),
                     bloomFilterFalsePositiveRate, mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this),
                     ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories,
                     filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory);
+                    metadataPageManagerFactory, serviceCtx.getTracer());
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 5fda514..22556af 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -59,7 +59,6 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
@@ -100,10 +99,10 @@
             IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, componentFactory, filterFrameFactory,
-                filterManager, filterFields, durable, filterHelper, invertedIndexFields, ITracer.NONE);
+                filterManager, filterFields, durable, filterHelper, invertedIndexFields, tracer);
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
@@ -111,7 +110,6 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
-
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             InMemoryInvertedIndex memInvIndex =
@@ -120,9 +118,8 @@
                     BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                             invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false);
-            LSMInvertedIndexMemoryComponent mutableComponent =
-                    new LSMInvertedIndexMemoryComponent(this, memInvIndex, deleteKeysBTree, virtualBufferCache,
-                            i == 0 ? true : false, filterHelper == null ? null : filterHelper.createFilter());
+            LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
+                    deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -296,7 +293,7 @@
         }
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, numBTreeTuples, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, numBTreeTuples, false, false, false);
 
         // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
         IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -349,65 +346,66 @@
         LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
         RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
         IIndexCursor cursor = mergeOp.getCursor();
-        try {
-            ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
-            // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
-            // Create an inverted index instance.
-            ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                    mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
-            ILSMDiskComponentBulkLoader componentBulkLoader;
-            // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
-            // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
-            if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
-                    .get(diskComponents.size() - 1)) {
-                // Keep the deleted tuples since the oldest disk component is not included in the merge operation
-                LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
-                        new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
-                try {
-                    long numElements = 0L;
-                    for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                        numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
-                                .getBloomFilter().getNumElements();
-                    }
-                    componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
-                    loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
-                } finally {
-                    btreeCursor.destroy();
-                }
-            } else {
-                componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
-            }
-            search(opCtx, cursor, mergePred);
+        ILSMIndexOperationContext opCtx = ((LSMInvertedIndexMergeCursor) cursor).getOpCtx();
+        // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+        // Create an inverted index instance.
+        ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
+                mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
+        ILSMDiskComponentBulkLoader componentBulkLoader;
+        // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
+        // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
+        if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+                .get(diskComponents.size() - 1)) {
+            // Keep the deleted tuples since the oldest disk component is not included in the merge operation
+            LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
+                    new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
             try {
-                while (cursor.hasNext()) {
-                    cursor.next();
-                    ITupleReference tuple = cursor.getTuple();
-                    componentBulkLoader.add(tuple);
-                }
-            } finally {
-                cursor.close();
-            }
-            if (component.getLSMComponentFilter() != null) {
-                List<ITupleReference> filterTuples = new ArrayList<>();
+                long numElements = 0L;
                 for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                    ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
-                    ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
-                    if (min != null) {
-                        filterTuples.add(min);
-                    }
-                    if (max != null) {
-                        filterTuples.add(max);
-                    }
+                    numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
+                            .getBloomFilter().getNumElements();
                 }
-                getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
-                        NoOpOperationCallback.INSTANCE);
-                getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+                componentBulkLoader =
+                        component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+                loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
+            } finally {
+                btreeCursor.destroy();
             }
-            componentBulkLoader.end();
-            return component;
-        } finally {
-            cursor.close();
+        } else {
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
+        search(opCtx, cursor, mergePred);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                componentBulkLoader.add(cursor.getTuple());
+            }
+        } finally {
+            try {
+                cursor.close();
+            } finally {
+                cursor.destroy();
+            }
+        }
+        if (component.getLSMComponentFilter() != null) {
+            List<ITupleReference> filterTuples = new ArrayList<>();
+            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
+                ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
+                if (min != null) {
+                    filterTuples.add(min);
+                }
+                if (max != null) {
+                    filterTuples.add(max);
+                }
+            }
+            getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
+                    NoOpOperationCallback.INSTANCE);
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+        }
+        componentBulkLoader.end();
+
+        return component;
     }
 
     private void loadDeleteTuples(ILSMIndexOperationContext opCtx,
@@ -488,16 +486,16 @@
             throws HyracksDataException {
         return new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(getHarness(), opCtx),
                 componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
     protected ILSMIOOperation createMergeOperation(AbstractLSMIndexOperationContext opCtx,
             LSMComponentFileReferences mergeFileRefs, ILSMIOOperationCallback callback) throws HyracksDataException {
         ILSMIndexAccessor accessor = new LSMInvertedIndexAccessor(getHarness(), opCtx);
-        IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(opCtx);
+        IIndexCursor cursor = new LSMInvertedIndexMergeCursor(opCtx);
         return new LSMInvertedIndexMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 247e44c..8e39b62 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -86,43 +85,42 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
-    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ctx.setOperation(IndexOperation.REPLICATE);
         ctx.getComponentsToBeReplicated().clear();
         ctx.getComponentsToBeReplicated().addAll(lsmComponents);
-        lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+        lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
index f1f5241..faa90eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
@@ -60,7 +60,6 @@
         ArrayList<IIndexAccessor> btreeAccessors = lsmInitialState.getDeletedKeysBTreeAccessors();
         for (int i = 0; i < numBTrees; i++) {
             rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(false);
-            btreeAccessors.get(i).search(rangeCursors[i], btreePredicate);
         }
         IndexCursorUtils.open(btreeAccessors, rangeCursors, btreePredicate);
         try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index 279a518..b030e83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -25,11 +25,14 @@
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class LSMInvertedIndexDiskComponent extends AbstractLSMWithBuddyDiskComponent {
@@ -109,4 +112,14 @@
         // Flush deleted keys BTree.
         ComponentUtils.markAsValid(getBuddyIndex(), persist);
     }
+
+    @Override
+    protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        IIndexBulkLoader indexBulkLoader =
+                invIndex.createMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        IIndexBulkLoader buddyBulkLoader =
+                getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
index 77c62b4..a8005bc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
@@ -20,9 +20,9 @@
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 
 public class LSMInvertedIndexMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -31,8 +31,8 @@
     private final BTree deletedKeysBTree;
 
     public LSMInvertedIndexMemoryComponent(LSMInvertedIndex lsmIndex, InMemoryInvertedIndex invIndex,
-            BTree deletedKeysBTree, IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            BTree deletedKeysBTree, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.invIndex = invIndex;
         this.deletedKeysBTree = deletedKeysBTree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
new file mode 100644
index 0000000..c80455d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueComparator;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueElement;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+/**
+ * This cursor is specially designed and optimized for merging inverted index.
+ * For simplicity, it assumes all components are disk components, and the cursor is not reused.
+ *
+ */
+public class LSMInvertedIndexMergeCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
+    protected final LSMInvertedIndexOpContext opCtx;
+    protected PriorityQueueElement outputTokenElement;
+    protected OnDiskInvertedIndexRangeSearchCursor[] rangeCursors;
+    protected PriorityQueueElement[] tokenQueueElements;
+    protected PriorityQueue<PriorityQueueElement> tokenQueue;
+    protected PriorityQueueComparator tokenQueueCmp;
+
+    protected PriorityQueueElement outputKeyElement;
+    protected PriorityQueueElement[] keyQueueElements;
+    protected PriorityQueue<PriorityQueueElement> keyQueue;
+    protected PriorityQueueComparator keyQueueCmp;
+
+    protected boolean needPushElementIntoKeyQueue;
+
+    protected ILSMHarness lsmHarness;
+
+    protected MultiComparator tokenCmp;
+    protected MultiComparator keyCmp;
+
+    protected List<ILSMComponent> operationalComponents;
+
+    // Assuming the cursor for all deleted-keys indexes are of the same type.
+    protected IIndexCursor[] deletedKeysBTreeCursors;
+    protected BloomFilter[] bloomFilters;
+    protected final long[] hashes = BloomFilter.createHashArray();
+    protected ArrayList<IIndexAccessor> deletedKeysBTreeAccessors;
+    protected RangePredicate deletedKeyBTreeSearchPred;
+
+    protected final TokenKeyPairTuple outputTuple;
+
+    public LSMInvertedIndexMergeCursor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = (LSMInvertedIndexOpContext) opCtx;
+        outputTokenElement = null;
+        outputKeyElement = null;
+        needPushElementIntoKeyQueue = false;
+
+        IInvertedIndex invertedIndex = (IInvertedIndex) this.opCtx.getIndex();
+        this.outputTuple = new TokenKeyPairTuple(invertedIndex.getTokenTypeTraits().length,
+                invertedIndex.getInvListTypeTraits().length);
+
+        this.tokenCmp = MultiComparator.create(invertedIndex.getTokenCmpFactories());
+        this.keyCmp = MultiComparator.create(invertedIndex.getInvListCmpFactories());
+        this.tokenQueueCmp = new PriorityQueueComparator(tokenCmp);
+        this.keyQueueCmp = new PriorityQueueComparator(keyCmp);
+    }
+
+    public LSMInvertedIndexOpContext getOpCtx() {
+        return opCtx;
+    }
+
+    @Override
+    public void doOpen(ICursorInitialState initState, ISearchPredicate searchPred) throws HyracksDataException {
+        LSMInvertedIndexRangeSearchCursorInitialState lsmInitState =
+                (LSMInvertedIndexRangeSearchCursorInitialState) initState;
+        int numComponents = lsmInitState.getNumComponents();
+        rangeCursors = new OnDiskInvertedIndexRangeSearchCursor[numComponents];
+        for (int i = 0; i < numComponents; i++) {
+            IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) lsmInitState.getIndexAccessors().get(i);
+            rangeCursors[i] = (OnDiskInvertedIndexRangeSearchCursor) invIndexAccessor.createRangeSearchCursor();
+            invIndexAccessor.rangeSearch(rangeCursors[i], lsmInitState.getSearchPredicate());
+        }
+        lsmHarness = lsmInitState.getLSMHarness();
+        operationalComponents = lsmInitState.getOperationalComponents();
+        deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
+        bloomFilters = new BloomFilter[deletedKeysBTreeAccessors.size()];
+        if (!deletedKeysBTreeAccessors.isEmpty()) {
+            deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
+            for (int i = 0; i < operationalComponents.size(); i++) {
+                ILSMComponent component = operationalComponents.get(i);
+                deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false);
+                if (component.getType() == LSMComponentType.MEMORY) {
+                    // No need for a bloom filter for the in-memory BTree.
+                    bloomFilters[i] = null;
+                } else {
+                    bloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
+                }
+            }
+        }
+        deletedKeyBTreeSearchPred = new RangePredicate(null, null, true, true, keyCmp, keyCmp);
+        initPriorityQueues();
+    }
+
+    private void initPriorityQueues() throws HyracksDataException {
+        int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
+        tokenQueue = new PriorityQueue<>(pqInitSize, tokenQueueCmp);
+        keyQueue = new PriorityQueue<>(pqInitSize, keyQueueCmp);
+        tokenQueueElements = new PriorityQueueElement[pqInitSize];
+        keyQueueElements = new PriorityQueueElement[pqInitSize];
+        for (int i = 0; i < pqInitSize; i++) {
+            tokenQueueElements[i] = new PriorityQueueElement(i);
+            keyQueueElements[i] = new PriorityQueueElement(i);
+        }
+        for (int i = 0; i < rangeCursors.length; i++) {
+            if (rangeCursors[i].hasNext()) {
+                rangeCursors[i].next();
+                tokenQueueElements[i].reset(rangeCursors[i].getTuple());
+                tokenQueue.offer(tokenQueueElements[i]);
+            } else {
+                rangeCursors[i].close();
+            }
+        }
+        searchNextToken();
+    }
+
+    private void searchNextToken() throws HyracksDataException {
+        if (tokenQueue.isEmpty()) {
+            return;
+        }
+        if (!keyQueue.isEmpty()) {
+            throw new IllegalStateException("Illegal call of initializing key queue");
+        }
+        outputTokenElement = tokenQueue.poll();
+        initPushIntoKeyQueue(outputTokenElement);
+        ITupleReference tokenTuple = getTokenTuple(outputTokenElement);
+        outputTuple.setTokenTuple(tokenTuple);
+        // pop all same tokens
+        while (!tokenQueue.isEmpty()) {
+            PriorityQueueElement tokenElement = tokenQueue.peek();
+            if (TupleUtils.equalTuples(tokenTuple, getTokenTuple(tokenElement), tokenCmp.getKeyFieldCount())) {
+                initPushIntoKeyQueue(tokenElement);
+                tokenQueue.poll();
+            } else {
+                break;
+            }
+        }
+    }
+
+    private ITupleReference getKeyTuple(PriorityQueueElement tokenElement) {
+        return ((TokenKeyPairTuple) tokenElement.getTuple()).getKeyTuple();
+    }
+
+    private ITupleReference getTokenTuple(PriorityQueueElement tokenElement) {
+        return ((TokenKeyPairTuple) tokenElement.getTuple()).getTokenTuple();
+    }
+
+    private void initPushIntoKeyQueue(PriorityQueueElement tokenElement) {
+        PriorityQueueElement keyElement = keyQueueElements[tokenElement.getCursorIndex()];
+        keyElement.reset(getKeyTuple(tokenElement));
+        keyQueue.add(keyElement);
+    }
+
+    private void pushIntoKeyQueueAndReplace(PriorityQueueElement keyElement) throws HyracksDataException {
+        int cursorIndex = keyElement.getCursorIndex();
+        if (rangeCursors[cursorIndex].hasNext()) {
+            rangeCursors[cursorIndex].next();
+            TokenKeyPairTuple tuple = (TokenKeyPairTuple) rangeCursors[cursorIndex].getTuple();
+            if (tuple.isNewToken()) {
+                // if this element is a new token, then the current inverted list has exuasted
+                PriorityQueueElement tokenElement = tokenQueueElements[cursorIndex];
+                tokenElement.reset(tuple);
+                tokenQueue.offer(tokenElement);
+            } else {
+                keyElement.reset(tuple.getKeyTuple());
+                keyQueue.offer(keyElement);
+            }
+        } else {
+            rangeCursors[cursorIndex].close();
+        }
+    }
+
+    @Override
+    public boolean doHasNext() throws HyracksDataException {
+        checkPriorityQueue();
+        return !keyQueue.isEmpty();
+    }
+
+    @Override
+    public void doNext() throws HyracksDataException {
+        outputKeyElement = keyQueue.poll();
+        outputTuple.setKeyTuple(outputKeyElement.getTuple());
+        needPushElementIntoKeyQueue = true;
+    }
+
+    @Override
+    public ITupleReference doGetTuple() {
+        return outputTuple;
+    }
+
+    protected void checkPriorityQueue() throws HyracksDataException {
+        checkKeyQueue();
+        if (keyQueue.isEmpty()) {
+            // if key queue is empty, we search the next token and check again
+            searchNextToken();
+            checkKeyQueue();
+        }
+    }
+
+    protected void checkKeyQueue() throws HyracksDataException {
+        while (!keyQueue.isEmpty() || needPushElementIntoKeyQueue) {
+            if (!keyQueue.isEmpty()) {
+                PriorityQueueElement checkElement = keyQueue.peek();
+                // If there is no previous tuple or the previous tuple can be ignored
+                if (outputKeyElement == null) {
+                    if (isDeleted(checkElement)) {
+                        // If the key has been deleted then pop it and set needPush to true.
+                        // We cannot push immediately because the tuple may be
+                        // modified if hasNext() is called
+                        outputKeyElement = checkElement;
+                        needPushElementIntoKeyQueue = true;
+                    } else {
+                        // we have found the next record
+                        return;
+                    }
+                } else {
+                    // Compare the previous tuple and the head tuple in the PQ
+                    if (keyCmp.compare(outputKeyElement.getTuple(), checkElement.getTuple()) == 0) {
+                        // If the previous tuple and the head tuple are
+                        // identical
+                        // then pop the head tuple and push the next tuple from
+                        // the tree of head tuple
+
+                        // the head element of PQ is useless now
+                        PriorityQueueElement e = keyQueue.poll();
+                        pushIntoKeyQueueAndReplace(e);
+                    } else {
+                        // If the previous tuple and the head tuple are different
+                        // the info of previous tuple is useless
+                        if (needPushElementIntoKeyQueue) {
+                            pushIntoKeyQueueAndReplace(outputKeyElement);
+                            needPushElementIntoKeyQueue = false;
+                        }
+                        outputKeyElement = null;
+                    }
+                }
+            } else {
+                // the priority queue is empty and needPush
+                // NOSONAR: outputKeyElement is not null when needPushElementIntoKeyQueue = true
+                pushIntoKeyQueueAndReplace(outputKeyElement);
+                needPushElementIntoKeyQueue = false;
+                outputKeyElement = null;
+            }
+        }
+    }
+
+    /**
+     * Check deleted-keys BTrees whether they contain the key in the checkElement's tuple.
+     */
+    protected boolean isDeleted(PriorityQueueElement keyElement) throws HyracksDataException {
+        ITupleReference keyTuple = keyElement.getTuple();
+        int end = keyElement.getCursorIndex();
+        for (int i = 0; i < end; i++) {
+            if (bloomFilters[i] != null && !bloomFilters[i].contains(keyTuple, hashes)) {
+                continue;
+            }
+            deletedKeysBTreeCursors[i].close();
+            deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], deletedKeyBTreeSearchPred);
+            try {
+                if (deletedKeysBTreeCursors[i].hasNext()) {
+                    return true;
+                }
+            } finally {
+                deletedKeysBTreeCursors[i].close();
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        outputTokenElement = null;
+        outputKeyElement = null;
+        needPushElementIntoKeyQueue = false;
+        try {
+            if (rangeCursors != null) {
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    rangeCursors[i].close();
+                }
+            }
+        } finally {
+            if (lsmHarness != null) {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        try {
+            if (tokenQueue != null) {
+                tokenQueue.clear();
+            }
+            if (keyQueue != null) {
+                keyQueue.clear();
+            }
+            if (rangeCursors != null) {
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    if (rangeCursors[i] != null) {
+                        rangeCursors[i].destroy();
+                    }
+                }
+                rangeCursors = null;
+            }
+        } finally {
+            if (lsmHarness != null) {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public ITupleReference getFilterMinTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMaxTuple() {
+        return null;
+    }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index 12dc23f..020b788 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -98,10 +98,9 @@
         keysOnlyTuple.reset(checkElement.getTuple());
         int end = checkElement.getCursorIndex();
         for (int i = 0; i < end; i++) {
-            if (bloomFilters[i] != null && bloomFilters[i].contains(keysOnlyTuple, hashes)) {
+            if (bloomFilters[i] != null && !bloomFilters[i].contains(keysOnlyTuple, hashes)) {
                 continue;
             }
-            deletedKeysBTreeCursors[i].close();
             deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
             try {
                 if (deletedKeysBTreeCursors[i].hasNext()) {
@@ -114,4 +113,31 @@
         return false;
     }
 
+    @Override
+    public void doClose() throws HyracksDataException {
+        try {
+            super.doClose();
+        } finally {
+            if (deletedKeysBTreeCursors != null) {
+                for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+                    deletedKeysBTreeCursors[i].close();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        try {
+            super.doDestroy();
+        } finally {
+            if (deletedKeysBTreeCursors != null) {
+                for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+                    deletedKeysBTreeCursors[i].destroy();
+                }
+                deletedKeysBTreeCursors = null;
+            }
+        }
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 4d444b9..d39c601 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -59,6 +59,7 @@
     private List<IIndexAccessor> deletedKeysBTreeAccessors;
     private RangePredicate keySearchPred;
     private ILSMIndexOperationContext opCtx;
+    private boolean includeMemoryComponents;
 
     private List<ILSMComponent> operationalComponents;
     private ITupleReference currentTuple = null;
@@ -76,7 +77,7 @@
         accessorIndex = 0;
         this.searchPred = searchPred;
         this.searchCallback = lsmInitState.getSearchOperationCallback();
-
+        includeMemoryComponents = false;
         // For searching the deleted-keys BTrees.
         deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
         deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
@@ -87,6 +88,7 @@
             if (component.getType() == LSMComponentType.MEMORY) {
                 // No need for a bloom filter for the in-memory BTree.
                 deletedKeysBTreeBloomFilters[i] = null;
+                includeMemoryComponents = true;
             } else {
                 deletedKeysBTreeBloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
             }
@@ -121,7 +123,8 @@
         while (currentCursor.hasNext()) {
             currentCursor.next();
             currentTuple = currentCursor.getTuple();
-            resultOfSearchCallBackProceed = searchCallback.proceed(currentTuple);
+            resultOfSearchCallBackProceed =
+                    includeMemoryComponents && accessorIndex == 0 ? searchCallback.proceed(currentTuple) : true;
 
             if (!resultOfSearchCallBackProceed) {
                 // We assume that the underlying cursors materialize their results such that
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index d7482a8..604d374 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
 
 public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
 
@@ -52,12 +53,12 @@
             IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
                 bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
                 tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
                 ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
index da3f079..7f3d12f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
@@ -220,9 +220,11 @@
             // Assumption: processing inverted list takes time; so, we don't want to keep them on the buffer cache.
             // Rather, we utilize the assigned working memory (buffers).
             tmpBuffer = page.getBuffer();
-            tmpBuffer.rewind();
-            buffers.get(currentBufferIdx).rewind();
-            buffers.get(currentBufferIdx).put(tmpBuffer);
+
+            // Copies the entire content of the page to the current buffer in the working memory.
+            System.arraycopy(tmpBuffer.array(), 0, buffers.get(currentBufferIdx).array(), 0,
+                    buffers.get(currentBufferIdx).capacity());
+            buffers.get(currentBufferIdx).position(buffers.get(currentBufferIdx).capacity());
 
             currentBufferIdx++;
             bufferCache.unpin(page);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
index b033b99..2401c67 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
@@ -53,6 +53,7 @@
     protected ICachedPage page;
 
     protected boolean pinned;
+    protected int pinnedPageId = -1;
 
     public FixedSizeElementInvertedListScanCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields)
             throws HyracksDataException {
@@ -103,14 +104,18 @@
      */
     @Override
     public void loadPages() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
         if (currentPageId == endPageId) {
+            // inverted list exhausted, return
             return;
         }
         currentPageId++;
+        if (pinned && pinnedPageId == currentPageId) {
+            // already pinned, return
+            return;
+        }
+        unloadPages();
         page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
+        pinnedPageId = currentPageId;
         pinned = true;
     }
 
@@ -134,7 +139,6 @@
         // Deducts 1 since the startPage would be set to bufferCurrentPageId + 1 in loadPages().
         this.currentPageId = startPageId - 1;
         this.numPages = endPageId - startPageId + 1;
-        this.pinned = false;
     }
 
     @Override
@@ -158,16 +162,14 @@
 
     @Override
     public void doClose() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
+        // No op
+        // We allow the inverted list cursor to hold at most one page to avoid
+        // unnecessary pins
     }
 
     @Override
     public void doDestroy() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
+        unloadPages();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 2f4f1d6..c3c9c21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexSearchCursorInitialState;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -204,9 +205,9 @@
         try {
             if (ctx.getBtreeCursor().hasNext()) {
                 ctx.getBtreeCursor().next();
-                openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor);
+                openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor, ctx);
             } else {
-                LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+                LSMInvertedIndexSearchCursorInitialState initState = ctx.getCursorInitialState();
                 initState.setInvertedListInfo(0, 0, 0, 0);
                 listCursor.open(initState, null);
             }
@@ -215,8 +216,8 @@
         }
     }
 
-    public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor)
-            throws HyracksDataException {
+    public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor,
+            OnDiskInvertedIndexOpContext opCtx) throws HyracksDataException {
         int startPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListStartPageIdField),
                 btreeTuple.getFieldStart(invListStartPageIdField));
         int endPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListEndPageIdField),
@@ -225,35 +226,33 @@
                 btreeTuple.getFieldStart(invListStartOffField));
         int numElements = IntegerPointable.getInteger(btreeTuple.getFieldData(invListNumElementsField),
                 btreeTuple.getFieldStart(invListNumElementsField));
-        LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+        LSMInvertedIndexSearchCursorInitialState initState = opCtx.getCursorInitialState();
         initState.setInvertedListInfo(startPageId, endPageId, startOff, numElements);
         listCursor.open(initState, null);
     }
 
-    public final class OnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
-        private final ArrayTupleBuilder btreeTupleBuilder;
-        private final ArrayTupleReference btreeTupleReference;
-        private final IIndexBulkLoader btreeBulkloader;
+    public abstract class AbstractOnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
+        protected final ArrayTupleBuilder btreeTupleBuilder;
+        protected final ArrayTupleReference btreeTupleReference;
+        protected final IIndexBulkLoader btreeBulkloader;
 
-        private int currentInvListStartPageId;
-        private int currentInvListStartOffset;
-        private final ArrayTupleBuilder lastTupleBuilder;
-        private final ArrayTupleReference lastTuple;
+        protected int currentInvListStartPageId;
+        protected int currentInvListStartOffset;
+        protected final ArrayTupleBuilder lastTupleBuilder;
+        protected final ArrayTupleReference lastTuple;
 
-        private int currentPageId;
-        private ICachedPage currentPage;
-        private final MultiComparator tokenCmp;
-        private final MultiComparator invListCmp;
+        protected int currentPageId;
+        protected ICachedPage currentPage;
+        protected final MultiComparator invListCmp;
 
-        private final boolean verifyInput;
-        private final MultiComparator allCmp;
+        protected final boolean verifyInput;
+        protected final MultiComparator allCmp;
 
-        private final IFIFOPageQueue queue;
+        protected final IFIFOPageQueue queue;
 
-        public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+        public AbstractOnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
                 boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
             this.verifyInput = verifyInput;
-            this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
             this.invListCmp = MultiComparator.create(invListCmpFactories);
             if (verifyInput) {
                 allCmp = MultiComparator.create(btree.getComparatorFactories(), invListCmpFactories);
@@ -272,22 +271,15 @@
             queue = bufferCache.createFIFOQueue();
         }
 
-        public void pinNextPage() throws HyracksDataException {
+        protected void pinNextPage() throws HyracksDataException {
             queue.put(currentPage);
             currentPageId++;
             currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
         }
 
-        private void createAndInsertBTreeTuple() throws HyracksDataException {
+        protected void insertBTreeTuple() throws HyracksDataException {
             // Build tuple.
-            btreeTupleBuilder.reset();
             DataOutput output = btreeTupleBuilder.getDataOutput();
-            // Add key fields.
-            lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
-            for (int i = 0; i < numTokenFields; i++) {
-                btreeTupleBuilder.addField(lastTuple.getFieldData(i), lastTuple.getFieldStart(i),
-                        lastTuple.getFieldLength(i));
-            }
             // Add inverted-list 'pointer' value fields.
             try {
                 output.writeInt(currentInvListStartPageId);
@@ -304,77 +296,59 @@
             // Reset tuple reference and add it into the BTree load.
             btreeTupleReference.reset(btreeTupleBuilder.getFieldEndOffsets(), btreeTupleBuilder.getByteArray());
             btreeBulkloader.add(btreeTupleReference);
+            btreeTupleBuilder.reset();
         }
 
-        /**
-         * Assumptions:
-         * The first btree.getMultiComparator().getKeyFieldCount() fields in tuple
-         * are btree keys (e.g., a string token).
-         * The next invListCmp.getKeyFieldCount() fields in tuple are keys of the
-         * inverted list (e.g., primary key).
-         * Key fields of inverted list are fixed size.
-         */
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            boolean firstElement = lastTupleBuilder.getSize() == 0;
-            boolean startNewList = firstElement;
-            if (!firstElement) {
-                // If the current and the last token don't match, we start a new list.
-                lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
-                startNewList = tokenCmp.compare(tuple, lastTuple) != 0;
-            }
-            if (startNewList) {
-                if (!firstElement) {
-                    // Create entry in btree for last inverted list.
-                    createAndInsertBTreeTuple();
-                }
-                if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                    pinNextPage();
-                    invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
-                    if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                        throw new IllegalStateException("Failed to create first inverted list.");
-                    }
-                }
-                currentInvListStartPageId = currentPageId;
-                currentInvListStartOffset = invListBuilder.getPos();
-            } else {
-                if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
-                    // Duplicate inverted-list element.
-                    return;
-                }
-            }
-
-            // Append to current inverted list.
-            if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+        protected void startNewList(ITupleReference tokenTuple) throws HyracksDataException {
+            if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
                 pinNextPage();
                 invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
-                if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+                if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
+                    throw new IllegalStateException("Failed to create first inverted list.");
+                }
+            }
+            currentInvListStartPageId = currentPageId;
+            currentInvListStartOffset = invListBuilder.getPos();
+        }
+
+        protected void appendInvertedList(ITupleReference keyTuple, int startField) throws HyracksDataException {
+            if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
+                pinNextPage();
+                invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+                if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
                     throw new IllegalStateException(
                             "Failed to append element to inverted list after switching to a new page.");
                 }
             }
+        }
 
-            if (verifyInput && lastTupleBuilder.getSize() != 0) {
-                if (allCmp.compare(tuple, lastTuple) <= 0) {
-                    throw new HyracksDataException(
-                            "Input stream given to OnDiskInvertedIndex bulk load is not sorted.");
-                }
+        protected void verifyTuple(ITupleReference tuple) throws HyracksDataException {
+            if (lastTupleBuilder.getSize() > 0 && allCmp.compare(tuple, lastTuple) <= 0) {
+                HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
             }
+        }
 
-            // Remember last tuple by creating a copy.
-            // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
+        protected void saveLastTuple(ITupleReference tuple) throws HyracksDataException {
             lastTupleBuilder.reset();
             for (int i = 0; i < tuple.getFieldCount(); i++) {
                 lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
             }
+            lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
+        }
+
+        protected void copyTokenToBTreeTuple(ITupleReference tokenTuple) throws HyracksDataException {
+            for (int i = 0; i < numTokenFields; i++) {
+                btreeTupleBuilder.addField(tokenTuple.getFieldData(i), tokenTuple.getFieldStart(i),
+                        tokenTuple.getFieldLength(i));
+            }
         }
 
         @Override
         public void end() throws HyracksDataException {
-            // The last tuple builder is empty if add() was never called.
-            if (lastTupleBuilder.getSize() != 0) {
-                createAndInsertBTreeTuple();
+            if (btreeTupleBuilder.getSize() != 0) {
+                insertBTreeTuple();
             }
+
             btreeBulkloader.end();
 
             if (currentPage != null) {
@@ -392,6 +366,72 @@
         }
     }
 
+    public class OnDiskInvertedIndexMergeBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+        public OnDiskInvertedIndexMergeBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+            super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException {
+            TokenKeyPairTuple pairTuple = (TokenKeyPairTuple) tuple;
+            ITupleReference tokenTuple = pairTuple.getTokenTuple();
+            ITupleReference keyTuple = pairTuple.getKeyTuple();
+            boolean startNewList = pairTuple.isNewToken();
+            if (startNewList) {
+                if (btreeTupleBuilder.getSize() > 0) {
+                    insertBTreeTuple();
+                }
+                startNewList(tokenTuple);
+                copyTokenToBTreeTuple(tokenTuple);
+            }
+            appendInvertedList(keyTuple, 0);
+            if (verifyInput) {
+                verifyTuple(tuple);
+                saveLastTuple(tuple);
+            }
+        }
+    }
+
+    public class OnDiskInvertedIndexBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+        public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+            super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException {
+            boolean firstElement = btreeTupleBuilder.getSize() == 0;
+            boolean startNewList = firstElement;
+            if (!firstElement) {
+                // If the current and the last token don't match, we start a new list.
+                startNewList = !TupleUtils.equalTuples(tuple, lastTuple, numTokenFields);
+            }
+            if (startNewList) {
+                if (!firstElement) {
+                    // Create entry in btree for last inverted list.
+                    insertBTreeTuple();
+                }
+                startNewList(tuple);
+                copyTokenToBTreeTuple(tuple);
+            } else {
+                if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
+                    // Duplicate inverted-list element.
+                    return;
+                }
+            }
+            appendInvertedList(tuple, numTokenFields);
+            if (verifyInput) {
+                verifyTuple(tuple);
+            }
+
+            saveLastTuple(tuple);
+        }
+
+    }
+
     @Override
     public IBufferCache getBufferCache() {
         return bufferCache;
@@ -518,6 +558,12 @@
                 rootPageId);
     }
 
+    public IIndexBulkLoader createMergeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws HyracksDataException {
+        return new OnDiskInvertedIndexMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                rootPageId);
+    }
+
     @Override
     public void validate() throws HyracksDataException {
         btree.validate();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
index 267cc79..81bc46b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexSearchCursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -32,12 +33,13 @@
 public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
 
     private final RangePredicate btreePred = new RangePredicate(null, null, true, true, null, null);
-    private IIndexAccessor btreeAccessor;
-    private IIndexCursor btreeCursor;
-    private MultiComparator searchCmp;
+    private final IIndexAccessor btreeAccessor;
+    private final IIndexCursor btreeCursor;
+    private final MultiComparator searchCmp;
     // For prefix search on partitioned indexes.
     private MultiComparator prefixSearchCmp;
     private boolean destroyed = false;
+    private LSMInvertedIndexSearchCursorInitialState cursorInitialState;
 
     public OnDiskInvertedIndexOpContext(BTree btree) throws HyracksDataException {
         // TODO: Ignore opcallbacks for now.
@@ -96,4 +98,11 @@
             btreeCursor.destroy();
         }
     }
+
+    public LSMInvertedIndexSearchCursorInitialState getCursorInitialState() {
+        if (cursorInitialState == null) {
+            cursorInitialState = new LSMInvertedIndexSearchCursorInitialState();
+        }
+        return cursorInitialState;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index d9e7d34..11b483e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -24,10 +24,9 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
 import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -41,7 +40,7 @@
 
     private final BTree btree;
     private final IIndexAccessor btreeAccessor;
-    private final IInPlaceInvertedIndex invIndex;
+    private final OnDiskInvertedIndex invIndex;
     private final IIndexOperationContext opCtx;
     private final InvertedListCursor invListRangeSearchCursor;
     private boolean isInvListCursorOpen;
@@ -50,11 +49,11 @@
     private RangePredicate btreePred;
 
     private final PermutingTupleReference tokenTuple;
-    private ConcatenatingTupleReference concatTuple;
+    private final TokenKeyPairTuple resultTuple;
 
-    public OnDiskInvertedIndexRangeSearchCursor(IInPlaceInvertedIndex invIndex, IIndexOperationContext opCtx)
+    public OnDiskInvertedIndexRangeSearchCursor(OnDiskInvertedIndex invIndex, IIndexOperationContext opCtx)
             throws HyracksDataException {
-        this.btree = ((OnDiskInvertedIndex) invIndex).getBTree();
+        this.btree = invIndex.getBTree();
         this.btreeAccessor = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
         this.invIndex = invIndex;
         this.opCtx = opCtx;
@@ -65,7 +64,7 @@
         }
         tokenTuple = new PermutingTupleReference(fieldPermutation);
         btreeCursor = btreeAccessor.createSearchCursor(false);
-        concatTuple = new ConcatenatingTupleReference(2);
+        resultTuple = new TokenKeyPairTuple(invIndex.getTokenTypeTraits().length, btree.getCmpFactories().length);
         invListRangeSearchCursor = invIndex.createInvertedListRangeSearchCursor();
         isInvListCursorOpen = false;
     }
@@ -87,11 +86,7 @@
             return true;
         }
         // The current inverted-list-range-search cursor is exhausted.
-        try {
-            invListRangeSearchCursor.unloadPages();
-        } finally {
-            invListRangeSearchCursor.close();
-        }
+        invListRangeSearchCursor.close();
         isInvListCursorOpen = false;
         openInvListRangeSearchCursor();
         return isInvListCursorOpen;
@@ -100,23 +95,14 @@
     @Override
     public void doNext() throws HyracksDataException {
         invListRangeSearchCursor.next();
-        if (concatTuple.hasMaxTuples()) {
-            concatTuple.removeLastTuple();
-        }
-        concatTuple.addTuple(invListRangeSearchCursor.getTuple());
+        resultTuple.setKeyTuple(invListRangeSearchCursor.getTuple());
     }
 
     @Override
     public void doDestroy() throws HyracksDataException {
         try {
-            if (isInvListCursorOpen) {
-                try {
-                    invListRangeSearchCursor.unloadPages();
-                } finally {
-                    isInvListCursorOpen = false;
-                    invListRangeSearchCursor.destroy();
-                }
-            }
+            invListRangeSearchCursor.destroy();
+            isInvListCursorOpen = false;
         } finally {
             btreeCursor.destroy();
         }
@@ -125,14 +111,8 @@
     @Override
     public void doClose() throws HyracksDataException {
         try {
-            if (isInvListCursorOpen) {
-                try {
-                    invListRangeSearchCursor.unloadPages();
-                } finally {
-                    invListRangeSearchCursor.close();
-                }
-                isInvListCursorOpen = false;
-            }
+            invListRangeSearchCursor.close();
+            isInvListCursorOpen = false;
         } finally {
             btreeCursor.close();
         }
@@ -140,7 +120,7 @@
 
     @Override
     public ITupleReference doGetTuple() {
-        return concatTuple;
+        return resultTuple;
     }
 
     // Opens an inverted-list-scan cursor for the given tuple.
@@ -148,11 +128,11 @@
         if (btreeCursor.hasNext()) {
             btreeCursor.next();
             tokenTuple.reset(btreeCursor.getTuple());
-            invIndex.openInvertedListCursor(invListRangeSearchCursor, tokenTuple, opCtx);
+            invIndex.openInvertedListCursor(btreeCursor.getTuple(), invListRangeSearchCursor,
+                    (OnDiskInvertedIndexOpContext) opCtx);
             invListRangeSearchCursor.prepareLoadPages();
             invListRangeSearchCursor.loadPages();
-            concatTuple.reset();
-            concatTuple.addTuple(tokenTuple);
+            resultTuple.setTokenTuple(tokenTuple);
             isInvListCursorOpen = true;
         } else {
             isInvListCursorOpen = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index eff4f5a..8c6b386 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -117,7 +117,7 @@
                 short numTokens = ShortPointable.getShort(btreeTuple.getFieldData(PARTITIONING_NUM_TOKENS_FIELD),
                         btreeTuple.getFieldStart(PARTITIONING_NUM_TOKENS_FIELD));
                 InvertedListCursor invListCursor = partSearcher.getCachedInvertedListCursor();
-                openInvertedListCursor(btreeTuple, invListCursor);
+                openInvertedListCursor(btreeTuple, invListCursor, ctx);
                 invListPartitions.addInvertedListCursor(invListCursor, numTokens);
                 tokenExists = true;
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
new file mode 100644
index 0000000..102fe96
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.tuples;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class TokenKeyPairTuple implements ITupleReference {
+
+    private ITupleReference tokenTuple;
+    private ITupleReference keyTuple;
+
+    private final int tokenFieldCount;
+    private final int keyFieldCount;
+
+    private boolean newToken;
+
+    public TokenKeyPairTuple(int tokenFieldCount, int keyFieldCount) {
+        this.tokenFieldCount = tokenFieldCount;
+        this.keyFieldCount = keyFieldCount;
+
+    }
+
+    public void setTokenTuple(ITupleReference token) {
+        this.tokenTuple = token;
+        this.keyTuple = null;
+    }
+
+    public void setKeyTuple(ITupleReference key) {
+        newToken = this.keyTuple == null;
+        this.keyTuple = key;
+    }
+
+    public ITupleReference getTokenTuple() {
+        return tokenTuple;
+    }
+
+    public ITupleReference getKeyTuple() {
+        return keyTuple;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return tokenFieldCount + keyFieldCount;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldData(fieldIndex);
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldStart(fieldIndex);
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldLength(fieldIndex);
+    }
+
+    private ITupleReference getTuple(int fIdx) {
+        return fIdx < tokenFieldCount ? tokenTuple : keyTuple;
+    }
+
+    private int getFieldIndex(int fIdx) {
+        return fIdx < tokenFieldCount ? fIdx : fIdx - tokenFieldCount;
+    }
+
+    public boolean isNewToken() {
+        return newToken;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index ba54ed4..1e812b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -62,6 +62,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
 
 public class InvertedIndexUtils {
 
@@ -129,7 +130,7 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            IMetadataPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
                 invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -166,7 +167,7 @@
                 filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
                 ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -178,7 +179,7 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
                 invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -215,6 +216,6 @@
                 filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index e36abb4..c695a5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -97,9 +97,8 @@
             BTree memBTree = new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                     btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, btreeCmpFactories.length,
                     ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_b_" + i));
-            LSMRTreeMemoryComponent mutableComponent =
-                    new LSMRTreeMemoryComponent(this, memRTree, memBTree, virtualBufferCache, i == 0 ? true : false,
-                            filterHelper == null ? null : filterHelper.createFilter());
+            LSMRTreeMemoryComponent mutableComponent = new LSMRTreeMemoryComponent(this, memRTree, memBTree,
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
index 7b12250..f902153 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -30,7 +31,6 @@
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
@@ -40,8 +40,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -50,9 +48,10 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -323,12 +322,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
-
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -383,16 +376,13 @@
         version = 0;
     }
 
-    // Not supported
     @Override
     public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
         throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree");
     }
 
-    // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
     }
 
@@ -445,139 +435,106 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     // The bulk loader used for both initial loading and transaction
     // modifications
     public class LSMTwoPCRTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
-        private final IIndexBulkLoader rtreeBulkLoader;
-        private final BTreeBulkLoader btreeBulkLoader;
-        private final IIndexBulkLoader builder;
-        private boolean cleanedUpArtifacts = false;
-        private boolean isEmptyComponent = true;
-        private boolean endedBloomFilterLoad = false;
         private final boolean isTransaction;
+        private final LoadOperation loadOp;
+        private final ChainedLSMDiskComponentBulkLoader componentBulkLoader;
 
         public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component = createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+                        componentFileRefs.getDeleteIndexFileReference(),
+                        componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
 
-            // Create the three loaders
-            rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getIndex().createBulkLoader(fillFactor, verifyInput,
-                    numElementsHint, false);
-            btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBuddyIndex()
-                    .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            BloomFilterSpecification bloomFilterSpec =
-                    BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-            builder = ((LSMRTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
-                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
+            componentBulkLoader =
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
         public void add(ITupleReference tuple) throws HyracksDataException {
-            try {
-                rtreeBulkLoader.add(tuple);
-            } catch (Exception e) {
-                cleanupArtifacts();
-                throw e;
-            }
-            if (isEmptyComponent) {
-                isEmptyComponent = false;
-            }
-        }
-
-        // This is made public in case of a failure, it is better to delete all
-        // created artifacts.
-        public void cleanupArtifacts() throws HyracksDataException {
-            if (!cleanedUpArtifacts) {
-                cleanedUpArtifacts = true;
-                ((LSMRTreeDiskComponent) component).deactivateAndDestroy();
-            }
+            componentBulkLoader.add(tuple);
         }
 
         @Override
         public void end() throws HyracksDataException {
-            if (!cleanedUpArtifacts) {
-                if (!endedBloomFilterLoad) {
-                    builder.end();
-                    endedBloomFilterLoad = true;
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        component.markAsValid(durable);
+                        ioOpCallback.afterFinalize(loadOp);
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(component);
+                    }
                 }
-                rtreeBulkLoader.end();
-                btreeBulkLoader.end();
-                if (isEmptyComponent) {
-                    cleanupArtifacts();
-                } else if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
-                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
         @Override
         public void delete(ITupleReference tuple) throws HyracksDataException {
-            try {
-                btreeBulkLoader.add(tuple);
-                builder.add(tuple);
-            } catch (Exception e) {
-                cleanupArtifacts();
-                throw e;
-            }
-            if (isEmptyComponent) {
-                isEmptyComponent = false;
-            }
+            componentBulkLoader.delete(tuple);
         }
 
         @Override
-        public void abort() {
+        public void abort() throws HyracksDataException {
             try {
-                cleanupArtifacts();
-            } catch (Exception e) {
-
+                try {
+                    componentBulkLoader.abort();
+                } finally {
+                    ioOpCallback.afterFinalize(loadOp);
+                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
-
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then eventually be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                    true);
-        }
     }
 
     // The only change the the schedule merge is the method used to create the
     // opCtx. first line <- in schedule merge, we->
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         rctx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -587,10 +544,12 @@
                         (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1));
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), rctx, buddyBTreeFields);
         // create the merge operation.
-        LSMRTreeMergeOperation mergeOp = new LSMRTreeMergeOperation(accessor, cursor,
-                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
-                relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
-        ioScheduler.scheduleOperation(mergeOp);
+        LSMRTreeMergeOperation mergeOp =
+                new LSMRTreeMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
+                        relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 4510618..4566d32 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -134,8 +134,8 @@
                 rTreeTupleSorter.sort();
                 component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
                         flushOp.getBloomFilterTarget(), true);
-                componentBulkLoader =
-                        component.createBulkLoader(1.0f, false, numBTreeTuples.longValue(), false, false, false);
+                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
+                        false, false, false);
                 flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
                 // scan the memory BTree and bulk load delete tuples
                 flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -332,11 +332,12 @@
                                 .getNumElements();
                     }
                     componentBulkLoader =
-                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false, false);
                     mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
                 } else {
                     //no buddy-btree needed
-                    componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false);
                 }
                 //search old rtree components
                 while (cursor.hasNext()) {
@@ -423,7 +424,7 @@
         LSMRTreeAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
                 componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -433,6 +434,6 @@
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
index 2e1ba68..ef7b815 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
@@ -21,9 +21,9 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
 
 public class LSMRTreeMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -32,8 +32,8 @@
     private final BTree btree;
 
     public LSMRTreeMemoryComponent(AbstractLSMRTree lsmIndex, RTree rtree, BTree btree, IVirtualBufferCache vbc,
-            boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.rtree = rtree;
         this.btree = btree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index c79735f..b9e174f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -98,7 +98,8 @@
                 rtreeCursors[currentCursor].next();
                 ITupleReference currentTuple = rtreeCursors[currentCursor].getTuple();
                 // Call proceed() to do necessary operations before returning this tuple.
-                resultOfsearchCallbackProceed = searchCallback.proceed(currentTuple);
+                resultOfsearchCallbackProceed =
+                        currentCursor == 0 && includeMutableComponent ? searchCallback.proceed(currentTuple) : true;
                 btreeTuple.reset(rtreeCursors[currentCursor].getTuple());
                 boolean killerTupleFound = false;
                 for (int i = 0; i < currentCursor && !killerTupleFound; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index a3ba4b1..2757595 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -109,7 +109,7 @@
                 try {
                     memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
                     component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
-                    componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
+                    componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
                     // Since the LSM-RTree is used as a secondary assumption, the
                     // primary key will be the last comparator in the BTree comparators
                     rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
@@ -235,7 +235,7 @@
         ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, 0L, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -273,7 +273,7 @@
             throws HyracksDataException {
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -288,6 +288,6 @@
                 new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx, returnDeletedTuples);
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 7db65bd..05dc3ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -177,7 +177,7 @@
                 // reconcile() and complete() can be added later after considering the semantics.
 
                 // Call proceed() to do necessary operations before returning this tuple.
-                resultOfsearchCallBackProceed = searchCallback.proceed(diskRTreeTuple);
+                resultOfsearchCallBackProceed = true;
                 if (searchMemBTrees(diskRTreeTuple, numMemoryComponents)) {
                     // anti-matter tuple is NOT found
                     foundNext = true;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
deleted file mode 100644
index 1ca75cb..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.rtree.api;
-
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-
-public interface IGenericPrimitiveSerializerDeserializer<T> extends ISerializerDeserializer<T> {
-    public double getValue(byte[] bytes, int offset);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 18ced6d..4dcfa94 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -55,8 +55,9 @@
             boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
             byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
             throws HyracksDataException {
+        // TODO: predicate & limit pushdown not enabled for RTree yet
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
                 appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
         if (keyFields != null && keyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index fc54903..d6c954e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -35,7 +35,7 @@
     public void add(ITupleReference tuple) throws HyracksDataException;
 
     /**
-     * Finalize the bulk loading operation in the given context.
+     * Finalize the bulk loading operation in the given context and release all resources.
      *
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 1443bbc..f8facf5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -77,7 +77,7 @@
     private final Queue<BufferCacheHeaderHelper> headerPageCache = new ConcurrentLinkedQueue<>();
 
     //DEBUG
-    private Level fileOpsLevel = Level.DEBUG;
+    private static final Level fileOpsLevel = Level.DEBUG;
     private ArrayList<CachedPage> confiscatedPages;
     private Lock confiscateLock;
     private HashMap<CachedPage, StackTraceElement[]> confiscatedPagesOwner;
@@ -588,7 +588,7 @@
             fInfo = fileInfoMap.get(fileId);
         }
         if (fInfo == null) {
-            throw new HyracksDataException("No such file mapped");
+            throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileId);
         }
         return fInfo;
     }
@@ -609,31 +609,34 @@
         BufferedFileHandle fInfo = getFileInfo(cPage);
         // synchronize on fInfo to prevent the file handle from being deleted until the page is written.
         synchronized (fInfo) {
-            if (!fInfo.fileHasBeenDeleted()) {
-                ByteBuffer buf = cPage.buffer.duplicate();
-                final int totalPages = cPage.getFrameSizeMultiplier();
-                final int extraBlockPageId = cPage.getExtraBlockPageId();
-                final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
-                BufferCacheHeaderHelper header = checkoutHeaderHelper();
-                try {
-                    buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
-                    buf.position(0);
-                    long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
-                            getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)),
-                            header.prepareWrite(cPage, buf));
+            if (fInfo.fileHasBeenDeleted()) {
+                return;
+            }
+            ByteBuffer buf = cPage.buffer.duplicate();
+            final int totalPages = cPage.getFrameSizeMultiplier();
+            final int extraBlockPageId = cPage.getExtraBlockPageId();
+            final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
+            BufferCacheHeaderHelper header = checkoutHeaderHelper();
+            try {
+                buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
+                buf.position(0);
+                long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
+                        getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)), header.prepareWrite(cPage, buf));
 
-                    if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
-                            + getPageSizeWithHeader()) {
-                        throw new HyracksDataException("Failed to write completely: " + bytesWritten);
-                    }
-                } finally {
-                    returnHeaderHelper(header);
+                if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
+                        + getPageSizeWithHeader()) {
+                    throw new HyracksDataException("Failed to write completely: " + bytesWritten);
                 }
-                if (totalPages > 1 && !contiguousLargePages) {
-                    buf.limit(totalPages * pageSize);
-                    ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
-                }
-                assert buf.capacity() == (pageSize * totalPages);
+            } finally {
+                returnHeaderHelper(header);
+            }
+            if (totalPages > 1 && !contiguousLargePages) {
+                buf.limit(totalPages * pageSize);
+                ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
+            }
+            if (buf.capacity() != pageSize * totalPages) {
+                throw new IllegalStateException("Illegal number of bytes written, expected bytes written: "
+                        + pageSize * totalPages + " actual bytes writte: " + buf.capacity());
             }
         }
     }
@@ -786,11 +789,8 @@
         synchronized (fileInfoMap) {
             fileInfoMap.forEach((key, value) -> {
                 try {
-                    boolean fileHasBeenDeleted = value.fileHasBeenDeleted();
-                    sweepAndFlush(key, !fileHasBeenDeleted);
-                    if (!fileHasBeenDeleted) {
-                        ioManager.close(value.getFileHandle());
-                    }
+                    sweepAndFlush(key, true);
+                    ioManager.close(value.getFileHandle());
                 } catch (HyracksDataException e) {
                     if (LOGGER.isWarnEnabled()) {
                         LOGGER.log(Level.WARN, "Error flushing file id: " + key, e);
@@ -827,15 +827,15 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Opening file: " + fileRef + " in cache: " + this);
         }
-        int fileId;
+        int fileId = -1;
         synchronized (fileInfoMap) {
             if (fileMapManager.isMapped(fileRef)) {
                 fileId = fileMapManager.lookupFileId(fileRef);
             } else {
                 fileId = fileMapManager.registerFile(fileRef);
             }
-            openFile(fileId);
         }
+        openFile(fileId);
         return fileId;
     }
 
@@ -844,42 +844,60 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Opening file: " + fileId + " in cache: " + this);
         }
-        synchronized (fileInfoMap) {
-            BufferedFileHandle fInfo;
-            fInfo = fileInfoMap.get(fileId);
-            if (fInfo == null) {
-                boolean unreferencedFileFound = true;
-                while (fileInfoMap.size() >= maxOpenFiles && unreferencedFileFound) {
-                    // map is full, make room by cleaning up unreferenced files
-                    unreferencedFileFound = false;
-                    for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
-                        if (entry.getValue().getReferenceCount() <= 0) {
-                            int entryFileId = entry.getKey();
-                            boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
-                            sweepAndFlush(entryFileId, !fileHasBeenDeleted);
-                            if (!fileHasBeenDeleted) {
-                                ioManager.close(entry.getValue().getFileHandle());
-                            }
-                            fileInfoMap.remove(entryFileId);
-                            unreferencedFileFound = true;
-                            // for-each iterator is invalid because we changed
-                            // fileInfoMap
-                            break;
+        BufferedFileHandle fInfo = null;
+        try {
+            fInfo = getOrCreateFileHandle(fileId);
+            if (fInfo.getFileHandle() == null) {
+                // a new file
+                synchronized (fInfo) {
+                    // prevent concurrent opening of the same file
+                    if (fInfo.getFileHandle() == null) {
+                        if (fileInfoMap.size() > maxOpenFiles) {
+                            closeOpeningFiles(fInfo);
                         }
+                        // create, open, and map new file reference
+                        FileReference fileRef = fileMapManager.lookupFileName(fileId);
+                        IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                        fInfo.setFileHandle(fh);
                     }
                 }
-                if (fileInfoMap.size() >= maxOpenFiles) {
-                    throw new HyracksDataException("Could not open fileId " + fileId + ". Max number of files "
-                            + maxOpenFiles + " already opened and referenced.");
-                }
-                // create, open, and map new file reference
-                FileReference fileRef = fileMapManager.lookupFileName(fileId);
-                IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                        IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-                fInfo = new BufferedFileHandle(fileId, fh);
-                fileInfoMap.put(fileId, fInfo);
             }
             fInfo.incReferenceCount();
+        } catch (Exception e) {
+            removeFileInfo(fileId);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void closeOpeningFiles(BufferedFileHandle newFileHandle) throws HyracksDataException {
+        synchronized (fileInfoMap) {
+            boolean unreferencedFileFound = true;
+            while (fileInfoMap.size() > maxOpenFiles && unreferencedFileFound) {
+                // map is full, make room by cleaning up unreferenced files
+                unreferencedFileFound = false;
+                for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
+                    BufferedFileHandle fh = entry.getValue();
+                    if (fh != newFileHandle && fh.getReferenceCount() <= 0) {
+                        if (fh.getReferenceCount() < 0) {
+                            throw new IllegalStateException("Illegal reference count " + fh.getReferenceCount()
+                                    + " of file " + fh.getFileHandle().getFileReference());
+                        }
+                        int entryFileId = entry.getKey();
+                        sweepAndFlush(entryFileId, true);
+                        ioManager.close(entry.getValue().getFileHandle());
+                        fileInfoMap.remove(entryFileId);
+                        unreferencedFileFound = true;
+                        // for-each iterator is invalid because we changed
+                        // fileInfoMap
+                        break;
+                    }
+                }
+            }
+            if (fileInfoMap.size() > maxOpenFiles) {
+                throw new HyracksDataException("Could not open fileId " + newFileHandle.getFileId()
+                        + ". Max number of files " + maxOpenFiles + " already opened and referenced.");
+            }
         }
     }
 
@@ -977,15 +995,19 @@
 
     @Override
     public void deleteFile(FileReference fileRef) throws HyracksDataException {
+        boolean mapped = false;
+        int fileId = -1;
         synchronized (fileInfoMap) {
             if (fileMapManager.isMapped(fileRef)) {
-                int fileId = fileMapManager.lookupFileId(fileRef);
-                deleteFile(fileId);
-                return;
-            } else {
-                IoUtil.delete(fileRef);
+                mapped = true;
+                fileId = fileMapManager.lookupFileId(fileRef);
             }
         }
+        if (mapped) {
+            deleteFile(fileId);
+        } else {
+            IoUtil.delete(fileRef);
+        }
     }
 
     @Override
@@ -993,29 +1015,26 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Deleting file: " + fileId + " in cache: " + this);
         }
-        synchronized (fileInfoMap) {
-            sweepAndFlush(fileId, false);
-            BufferedFileHandle fInfo = null;
+        BufferedFileHandle fInfo = removeFileInfo(fileId);
+        if (fInfo == null) {
+            return;
+        }
+        sweepAndFlush(fileId, false);
+        try {
+            if (fInfo.getReferenceCount() > 0) {
+                throw new HyracksDataException("Deleting open file");
+            }
+        } finally {
+            FileReference fileRef = null;
             try {
-                fInfo = fileInfoMap.get(fileId);
-                if (fInfo != null && fInfo.getReferenceCount() > 0) {
-                    throw new HyracksDataException("Deleting open file");
+                synchronized (fileInfoMap) {
+                    fileRef = fileMapManager.unregisterFile(fileId);
                 }
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
             } finally {
-                FileReference fileRef = fileMapManager.unregisterFile(fileId);
                 try {
-                    if (fInfo != null) {
-                        // Mark the fInfo as deleted,
-                        // such that when its pages are reclaimed in openFile(),
-                        // the pages are not flushed to disk but only invalidated.
-                        synchronized (fInfo) {
-                            if (!fInfo.fileHasBeenDeleted()) {
-                                ioManager.close(fInfo.getFileHandle());
-                                fInfo.markAsDeleted();
-                            }
-                        }
+                    synchronized (fInfo) {
+                        ioManager.close(fInfo.getFileHandle());
+                        fInfo.markAsDeleted();
                     }
                 } finally {
                     IoUtil.delete(fileRef);
@@ -1264,6 +1283,18 @@
         return null;
     }
 
+    private BufferedFileHandle getOrCreateFileHandle(int fileId) {
+        synchronized (fileInfoMap) {
+            return fileInfoMap.computeIfAbsent(fileId, id -> new BufferedFileHandle(fileId, null));
+        }
+    }
+
+    private BufferedFileHandle removeFileInfo(int fileId) {
+        synchronized (fileInfoMap) {
+            return fileInfoMap.remove(fileId);
+        }
+    }
+
     private ICachedPage getPageLoop(long dpid, int multiplier, boolean confiscate) throws HyracksDataException {
         final long startingPinCount = DEBUG ? masterPinCount.get() : -1;
         int cycleCount = 0;
@@ -1404,14 +1435,14 @@
 
     @Override
     public void purgeHandle(int fileId) throws HyracksDataException {
-        synchronized (fileInfoMap) {
-            BufferedFileHandle fh = fileInfoMap.get(fileId);
-            if (fh != null) {
-                ioManager.close(fh.getFileHandle());
-                fileInfoMap.remove(fileId);
+        BufferedFileHandle fh = removeFileInfo(fileId);
+        if (fh != null) {
+            synchronized (fileInfoMap) {
                 fileMapManager.unregisterFile(fileId);
             }
+            ioManager.close(fh.getFileHandle());
         }
+
     }
 
     static class BufferCacheHeaderHelper {
@@ -1451,4 +1482,23 @@
             return multiplier;
         }
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        synchronized (fileInfoMap) {
+            if (fileMapManager.isMapped(fileRef)) {
+                int fileId;
+                try {
+                    fileId = fileMapManager.lookupFileId(fileRef);
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+                BufferedFileHandle fInfo = fileInfoMap.get(fileId);
+                if (fInfo != null && fInfo.getReferenceCount() > 0) {
+                    fInfo.decReferenceCount();
+                }
+            }
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 1cbe404..9e92f21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -246,4 +246,10 @@
         deleteFileCount.incrementAndGet();
         bufferCache.deleteFile(file);
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        bufferCache.closeFileIfOpen(fileRef);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
index 9d0b728..856edbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
@@ -15,6 +15,7 @@
 
 package org.apache.hyracks.storage.common.buffercache;
 
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class FIFOLocalWriter implements IFIFOPageWriter {
@@ -29,6 +30,10 @@
         CachedPage cPage = (CachedPage) page;
         try {
             bufferCache.write(cPage);
+        } catch (HyracksDataException e) {
+            if (e.getErrorCode() != ErrorCode.FILE_DOES_NOT_EXIST) {
+                throw HyracksDataException.create(e);
+            }
         } finally {
             bufferCache.returnPage(cPage);
             if (DEBUG) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 8dccc4a..21d3677 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -269,4 +269,13 @@
      */
     void resizePage(ICachedPage page, int multiplier, IExtraPageBlockHelper extraPageBlockHelper)
             throws HyracksDataException;
+
+    /**
+     * Close the file if open.
+     *
+     * @param fileRef
+     * @throws HyracksDataException
+     */
+    void closeFileIfOpen(FileReference fileRef);
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
index 177128e..1312d97 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
@@ -37,6 +37,10 @@
         return fileId;
     }
 
+    public void setFileHandle(IFileHandle fileHandle) {
+        this.handle = fileHandle;
+    }
+
     public IFileHandle getFileHandle() {
         return handle;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
deleted file mode 100644
index d303cca..0000000
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.common;
-
-import java.util.Collection;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.common.IIndexAccessor;
-
-@SuppressWarnings("rawtypes")
-public interface ITreeIndexTestContext<T extends CheckTuple> {
-    public int getFieldCount();
-
-    public int getKeyFieldCount();
-
-    public ISerializerDeserializer[] getFieldSerdes();
-
-    public IBinaryComparatorFactory[] getComparatorFactories();
-
-    public IIndexAccessor getIndexAccessor();
-
-    public ITreeIndex getIndex();
-
-    public ArrayTupleReference getTuple();
-
-    public ArrayTupleBuilder getTupleBuilder();
-
-    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples);
-
-    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples);
-
-    public Collection<T> getCheckTuples();
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
new file mode 100644
index 0000000..cb791c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class EncapsulatingIoCallback implements ILSMIOOperationCallback {
+
+    private final ILSMIOOperationCallback encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallback(ILSMIOOperationCallback inner, ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        this.encapsulated = inner;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        scheduledCallback.before(operation);
+        encapsulated.scheduled(operation);
+        scheduledCallback.after(operation);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperationCallback.before(operation);
+        encapsulated.beforeOperation(operation);
+        beforeOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperationCallback.before(operation);
+        encapsulated.afterOperation(operation);
+        afterOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalizeCallback.before(operation);
+        encapsulated.afterFinalize(operation);
+        afterFinalizeCallback.after(operation);
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        try {
+            completedCallback.before(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+        encapsulated.completed(operation);
+        try {
+            completedCallback.after(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.recycled(component);
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.allocated(component);
+    }
+
+    public ILSMIOOperationCallback getEncapsulated() {
+        return encapsulated;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
new file mode 100644
index 0000000..bb10236
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class EncapsulatingIoCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ILSMIOOperationCallbackFactory encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallbackFactory(ILSMIOOperationCallbackFactory factory,
+            ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        encapsulated = factory;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        encapsulated.initialize(ncCtx, resource);
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        ILSMIOOperationCallback inner = encapsulated.createIoOpCallback(index);
+        return new EncapsulatingIoCallback(inner, scheduledCallback, beforeOperationCallback, afterOperationCallback,
+                afterFinalizeCallback, completedCallback);
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return encapsulated.getCurrentMemoryComponentIndex();
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
new file mode 100644
index 0000000..21f3877
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -0,0 +1,470 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.NoOpTestCallback;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LSMBTreeComponentLifecycleTest {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private final OrderedIndexTestUtils testUtils = new OrderedIndexTestUtils();
+    private final ISerializerDeserializer[] fieldSerdes =
+            { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final int numKeys = 1;
+    private static final int numTuplesToInsert = 100;
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            ILSMIOOperationScheduler scheduler, ILSMIOOperationCallbackFactory ioCallbackFactory) throws Exception {
+        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
+                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                scheduler, ioCallbackFactory, harness.getMetadataPageManagerFactory(), false, true, false);
+    }
+
+    private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys)
+            throws Exception {
+        return createTestContext(fieldSerdes, numKeys, harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory());
+    }
+
+    @Test
+    public void testFlushUnallocatedIndex() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnallocatedIndexStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnallocatedIndex();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testNormalFlushOperation() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testNormalFlushOperationStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testNormalFlushOperation();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testFlushUnModifiedComponent() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnModifiedComponentStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnModifiedComponent();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    public int getExpectedMemoryComponentIndex(int expectedIndex) {
+        return (CountingIoOperationCallbackFactory.STARTING_INDEX + expectedIndex) % 2;
+    }
+
+    @Test
+    public void testScheduleMoreFlushesThanComponents() throws Exception {
+        final AtomicInteger counter = new AtomicInteger();
+        final Semaphore flushSemaphore = new Semaphore(0);
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, new AsynchronousScheduler(
+                r -> new Thread(r, "LsmIoThread-" + counter.getAndIncrement()), new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Scheduler failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Operation {} failed", operation, failure);
+                    }
+                }), new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
+                        NoOpTestCallback.get(), new ITestOpCallback<ILSMIOOperation>() {
+                            @Override
+                            public void before(ILSMIOOperation t) throws HyracksDataException {
+                                try {
+                                    flushSemaphore.acquire();
+                                } catch (InterruptedException e) {
+                                    throw new IllegalStateException(e);
+                                }
+                            }
+
+                            @Override
+                            public void after(ILSMIOOperation t) throws HyracksDataException {
+                            }
+                        }, NoOpTestCallback.get(), NoOpTestCallback.get()));
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        int numMemoryComponents = index.getNumberOfAllMemoryComponents();
+        // create a flusher that will schedule 13 flushes.
+        // wait for all flushes to be scheduled.
+        // create an inserter that will insert some records.
+        // one by one allow flushes until one flush remains, and ensure no record went in.
+        // allow the last flush, then wait for the inserts to succeed, and ensure they went to
+        // the expected memory component
+        final int numFlushes = 13;
+        User firstUser = new User("FirstUser");
+        User secondUser = new User("SecondUser");
+        Request flushRequest = new Request(Request.Statement.FLUSH, ctx, numFlushes);
+        firstUser.add(flushRequest);
+        firstUser.step();
+        // wait until all flushes have been scheduled.. Not yet performed
+        flushRequest.await(1);
+        // create an inserter and allow it to go all the way
+        Request insertRequest = new Request(Request.Statement.INSERT, ctx, 1);
+        secondUser.add(insertRequest);
+        secondUser.step();
+        secondUser.step();
+        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        ILSMIndexOperationContext opCtx = accessor.getOpContext();
+        assertCorrectSearchComponents(opCtx, index, 0);
+        // Allow one flush at a time and ensure that inserter didn't succeed
+        for (int i = 0; i < numFlushes - 1; i++) {
+            flushSemaphore.release();
+            firstUser.step();
+            flushRequest.await(2 + i);
+            Assert.assertEquals(0, insertRequest.getSteps());
+            // also ensure that you get the correct components when searching
+            assertCorrectSearchComponents(opCtx, index, i + 1);
+        }
+        // Allow last flush to proceed
+        flushSemaphore.release();
+        // wait for the insert to complete
+        insertRequest.await();
+        firstUser.step();
+        firstUser.step();
+        flushRequest.await();
+        firstUser.stop();
+        secondUser.stop();
+
+        int expectedMemoryComponent = numFlushes % numMemoryComponents;
+        Assert.assertEquals(getExpectedMemoryComponentIndex(expectedMemoryComponent),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+
+        EncapsulatingIoCallback encapsulating = (EncapsulatingIoCallback) index.getIOOperationCallback();
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) encapsulating.getEncapsulated();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(numFlushes, ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flushSemaphore.release();
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex((expectedMemoryComponent + 1) % numMemoryComponents),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // deactivate will cause a flush
+        flushSemaphore.release();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    private void assertCorrectSearchComponents(ILSMIndexOperationContext opCtx, ILSMIndex index,
+            int numSuccesfullyCompletedFlushes) throws HyracksDataException {
+        opCtx.reset();
+        opCtx.setOperation(IndexOperation.SEARCH);
+        index.getOperationalComponents(opCtx);
+        List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
+        int first = numSuccesfullyCompletedFlushes % memComponents.size();
+        Assert.assertEquals(memComponents.get(first), getFirstMemoryComponent(opCtx));
+    }
+
+    private ILSMComponent getFirstMemoryComponent(ILSMIndexOperationContext opCtx) {
+        List<ILSMComponent> components = opCtx.getComponentHolder();
+        // backward
+        for (int i = components.size() - 1; i >= 0; i--) {
+            ILSMComponent next = components.get(i);
+            if (next.getType() == LSMComponentType.MEMORY) {
+                return next;
+            }
+        }
+        return null;
+    }
+
+    private void flush(OrderedIndexTestContext ctx) throws HyracksDataException, InterruptedException {
+        ILSMIOOperation flush = scheduleFlush(ctx);
+        flush.sync();
+        if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(flush.getFailure());
+        }
+    }
+
+    private ILSMIOOperation scheduleFlush(OrderedIndexTestContext ctx)
+            throws HyracksDataException, InterruptedException {
+        ILSMIndexAccessor accessor =
+                (ILSMIndexAccessor) ctx.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        return accessor.scheduleFlush();
+    }
+
+    private static class Request {
+        private enum Statement {
+            FLUSH,
+            INSERT
+        }
+
+        private final Statement statement;
+        private final OrderedIndexTestContext ctx;
+        private final int repeats;
+        private boolean done = false;
+        private int step = 0;
+
+        public Request(Statement statement, OrderedIndexTestContext ctx, int repeats) {
+            this.statement = statement;
+            this.ctx = ctx;
+            this.repeats = repeats;
+        }
+
+        Statement statement() {
+            return statement;
+        }
+
+        synchronized void complete() {
+            done = true;
+            notifyAll();
+        }
+
+        synchronized void await() throws InterruptedException {
+            while (!done) {
+                wait();
+            }
+        }
+
+        synchronized void step() {
+            step++;
+            notifyAll();
+        }
+
+        synchronized int getSteps() {
+            return step;
+        }
+
+        synchronized void await(int step) throws InterruptedException {
+            while (this.step < step) {
+                wait();
+            }
+        }
+    }
+
+    private class User extends SingleThreadEventProcessor<Request> {
+
+        private Semaphore step = new Semaphore(0);
+
+        public User(String username) {
+            super(username);
+        }
+
+        public void step() {
+            step.release();
+        }
+
+        @Override
+        protected void handle(Request req) throws Exception {
+            try {
+                step.acquire();
+                switch (req.statement()) {
+                    case FLUSH:
+                        List<ILSMIOOperation> flushes = new ArrayList<>(req.repeats);
+                        for (int i = 0; i < req.repeats; i++) {
+                            flushes.add(scheduleFlush(req.ctx));
+                        }
+                        req.step();
+                        for (ILSMIOOperation op : flushes) {
+                            step.acquire();
+                            op.sync();
+                            if (op.getStatus() == LSMIOOperationStatus.FAILURE) {
+                                throw HyracksDataException.create(op.getFailure());
+                            }
+                            req.step(); // report after completion of each flush
+                        }
+                        break;
+                    case INSERT:
+                        testUtils.insertIntTuples(req.ctx, numTuplesToInsert, harness.getRandom());
+                        break;
+                    default:
+                        break;
+                }
+                req.step();
+                step.acquire();
+            } finally {
+                req.step();
+                req.complete();
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
index 9d768aa..03b221e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
@@ -29,7 +29,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.junit.After;
 import org.junit.Assert;
@@ -71,8 +70,8 @@
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
         accessor.insert(tuple);
 
-        // Flush to generate a disk component
-        accessor.scheduleFlush(((ILSMIndex) ctx.getIndex()).getIOOperationCallback());
+        // Flush to generate a disk component. This uses synchronous scheduler
+        accessor.scheduleFlush();
 
         // Make sure the disk component was generated
         LSMBTree btree = (LSMBTree) ctx.getIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
index 815c8bc..1b9a7a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
@@ -40,9 +40,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.hyracks.storage.am.lsm.common.impls.StubIOOperationCallback;
 import org.junit.Assert;
 
 /**
@@ -113,13 +113,14 @@
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getRight(), minMax.getRight(), comp));
                 }
 
-                StubIOOperationCallback stub = new StubIOOperationCallback();
-                BlockingIOOperationCallbackWrapper waiter = new BlockingIOOperationCallbackWrapper(stub);
-                accessor.scheduleFlush(waiter);
-                waiter.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 if (minMax != null) {
                     Pair<ITupleReference, ITupleReference> obsMinMax =
-                            filterToMinMax(stub.getLastNewComponent().getLSMComponentFilter());
+                            filterToMinMax(flush.getNewComponent().getLSMComponentFilter());
                     Assert.assertEquals(0,
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getLeft(), minMax.getLeft(), comp));
                     Assert.assertEquals(0,
@@ -143,8 +144,7 @@
                     expectedMergeMinMax.setRight(componentMinMax.getRight());
                 }
             }
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             flushedComponents = ((LSMBTree) ctx.getIndex()).getDiskComponents();
             Pair<ITupleReference, ITupleReference> mergedMinMax =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index 475ab9c..a8639c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -76,32 +77,24 @@
         ITupleReference tuple1 = TupleUtils.createIntegerTuple(1, 1, 1);
         accessor.insert(tuple1);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple2 = TupleUtils.createIntegerTuple(2, 2, 2);
         accessor.insert(tuple2);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple3 = TupleUtils.createIntegerTuple(3, 3, 3);
         accessor.insert(tuple3);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
-
+        accessor.scheduleFlush();
         scheduler.modify = true;
-
-        boolean exceptionThrown = false;
-        try {
-            accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
-        } catch (HyracksDataException e) {
-            exceptionThrown = true;
-        }
-        Assert.assertTrue(exceptionThrown);
-
+        ILSMIOOperation merge = accessor.scheduleMerge(btree.getDiskComponents());
+        merge.sync();
+        Assert.assertEquals(LSMIOOperationStatus.FAILURE, merge.getStatus());
         scheduler.modify = false;
-        accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
+        accessor.scheduleMerge(btree.getDiskComponents());
         Assert.assertEquals(1, btree.getDiskComponents().size());
-
         btree.deactivate();
         btree.destroy();
     }
@@ -120,14 +113,18 @@
 
         @Override
         public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-            if (modify) {
-                try {
-                    modifyOperation(operation);
-                } catch (Exception e) {
-                    throw HyracksDataException.create(e);
+            try {
+                if (modify) {
+                    try {
+                        modifyOperation(operation);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
                 }
+                operation.call();
+            } finally {
+                operation.complete();
             }
-            operation.call();
         }
 
         private void modifyOperation(ILSMIOOperation operation) throws Exception {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 87ddf33..7c59671 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -74,8 +74,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             orderedIndexTestUtils.checkPointSearches(ctx);
             orderedIndexTestUtils.checkScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 9f17efa..4fafb38 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.btree;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
@@ -26,9 +27,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.Test;
@@ -72,8 +73,6 @@
         IndexAccessParameters actx = new IndexAccessParameters(cb, NoOpOperationCallback.INSTANCE);
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(actx);
 
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             isFoundNull = true;
             for (int i = 0; i < NUM_TUPLES; i++) {
@@ -82,8 +81,11 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -95,8 +97,7 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                accessor.scheduleFlush().sync();
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -106,9 +107,7 @@
                 TupleUtils.createIntegerTuple(builder, tuple, i);
                 accessor.delete(tuple);
             }
-
-            accessor.scheduleFlush(ioOpCallback);
-            ioOpCallback.waitForIO();
+            accessor.scheduleFlush().sync();
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index 48812b4..2574c4d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -44,8 +44,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -113,21 +111,20 @@
     protected void test(OrderedIndexTestContext ctx, ISerializerDeserializer[] fieldSerdes)
             throws HyracksDataException {
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         //component 2 contains 1 and 2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 1 contains 1 and -2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         deleteTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 0 contains 2 and 3
         upsertTuple(ctx, fieldSerdes, getValue(3, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 3, btree.getDiskComponents().size());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index 3a48160..afcb3c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -52,8 +52,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -352,11 +350,10 @@
             throws HyracksDataException {
 
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         op1.performOperation(ctx, AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT);
         op2.performOperation(ctx,
                 AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT / AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS);
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 1, btree.getDiskComponents().size());
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index fc852cd..0914541 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -35,9 +35,9 @@
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
@@ -99,8 +99,6 @@
 
     private void test(IndexModification op1, IndexModification op2) throws Exception {
         ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) accessor;
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             index.clear();
             isFoundNull = true;
@@ -111,8 +109,11 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = lsmAccessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
                 isUpdated = false;
             } else {
@@ -126,8 +127,7 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                lsmAccessor.scheduleFlush().sync();
             } else {
                 isFoundNull = false;
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
index 19a9872..2e43198 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
@@ -35,7 +35,7 @@
     }
 
     @Override
-    public void after() throws HyracksDataException {
+    public void after(Semaphore t) throws HyracksDataException {
 
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
new file mode 100644
index 0000000..84b530a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class CountingIoOperationCallback implements ILSMIOOperationCallback {
+    private int beforeOperation;
+    private int afterOperation;
+    private int afterFinalize;
+    private int recycled;
+    private int allocated;
+    private int beforeSchedule;
+    private int destroy;
+
+    public CountingIoOperationCallback() {
+    }
+
+    public int getAfterFinalizeCount() {
+        return afterFinalize;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        recycled++;
+    }
+
+    public int getRecycledCount() {
+        return recycled;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        allocated++;
+    }
+
+    public int getAllocatedCount() {
+        return allocated;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        beforeSchedule++;
+    }
+
+    public int getBeforeScheduleCount() {
+        return beforeSchedule;
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperation++;
+    }
+
+    public int getBeforeOperationCount() {
+        return beforeOperation;
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperation++;
+    }
+
+    public int getAfterOperationCount() {
+        return afterOperation;
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalize++;
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        destroy++;
+    }
+
+    public int getDestroyCount() {
+        return destroy;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
new file mode 100644
index 0000000..b64b9b1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class CountingIoOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static int STARTING_INDEX = 0;
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        // No op
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        return new CountingIoOperationCallback();
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return STARTING_INDEX;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
index e888238..fa3097d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
@@ -23,5 +23,5 @@
 public interface ITestOpCallback<T> {
     void before(T t) throws HyracksDataException;
 
-    void after() throws HyracksDataException;
+    void after(T t) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
new file mode 100644
index 0000000..2370663
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class NoOpTestCallback implements ITestOpCallback {
+    public static final NoOpTestCallback INSTANCE = new NoOpTestCallback();
+
+    private NoOpTestCallback() {
+    }
+
+    @SuppressWarnings("unchecked")
+    public static final <T> ITestOpCallback<T> get() {
+        return INSTANCE;
+    }
+
+    @Override
+    public void before(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+    @Override
+    public void after(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
index 1d4b7d6..4867c71 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -65,9 +64,11 @@
 
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioAllocateCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioRecycleCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpScheduledCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioBeforeCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterOpCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterFinalizeCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpCompletedCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> allocateComponentCallbacks = new ArrayList<>();
 
     private volatile int numScheduledFlushes;
@@ -110,7 +111,7 @@
         super.modify(ictx, tuple);
         synchronized (modifyCallbacks) {
             for (ITestOpCallback<Semaphore> callback : modifyCallbacks) {
-                callback.after();
+                callback.after(modifySemaphore);
             }
         }
     }
@@ -122,17 +123,15 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleFlush(ctx, callback);
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledFlushes++;
+        return super.createFlushOperation(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleMerge(ctx, callback);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledMerges++;
+        return super.createMergeOperation(ctx);
     }
 
     @Override
@@ -148,7 +147,7 @@
         numFinishedFlushes++;
         synchronized (flushCallbacks) {
             for (ITestOpCallback<Semaphore> callback : flushCallbacks) {
-                callback.after();
+                callback.after(flushSemaphore);
             }
         }
         return c;
@@ -167,7 +166,7 @@
         numFinishedMerges++;
         synchronized (mergeCallbacks) {
             for (ITestOpCallback<Semaphore> callback : mergeCallbacks) {
-                callback.after();
+                callback.after(mergeSemaphore);
             }
         }
         return c;
@@ -347,6 +346,30 @@
         }
     }
 
+    public void addIoScheduledCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoScheduledCallbacks() {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.clear();
+        }
+    }
+
+    public void addIoCompletedCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoCompletedCallbacks() {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.clear();
+        }
+    }
+
     @Override
     public void allocateMemoryComponents() throws HyracksDataException {
         synchronized (allocateComponentCallbacks) {
@@ -357,7 +380,7 @@
         super.allocateMemoryComponents();
         synchronized (allocateComponentCallbacks) {
             for (ITestOpCallback<Void> callback : allocateComponentCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -373,7 +396,7 @@
     public void beforeIoOperationReturned() throws HyracksDataException {
         synchronized (ioBeforeCallbacks) {
             for (ITestOpCallback<Void> callback : ioBeforeCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -389,7 +412,7 @@
     public void afterIoOperationReturned() throws HyracksDataException {
         synchronized (ioAfterOpCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterOpCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -405,7 +428,39 @@
     public void afterIoFinalizeReturned() throws HyracksDataException {
         synchronized (ioAfterFinalizeCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterFinalizeCallbacks) {
-                callback.after();
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioScheduledCalled() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioScheduledReturned() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioCompletedCalled() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioCompletedReturned() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.after(null);
             }
         }
     }
@@ -421,7 +476,7 @@
     public void recycledReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioRecycleCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioRecycleCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
@@ -437,7 +492,7 @@
     public void allocatedReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioAllocateCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioAllocateCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index c7e064f..2e7a9f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -212,4 +212,10 @@
     public IFileMapManager getFileMapProvider() {
         return vbc.getFileMapProvider();
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 4c07dea..583fd5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -116,7 +116,7 @@
                 consumeCursorTuples(searchCursor);
                 break;
             case MERGE:
-                accessor.scheduleMerge(lsmBTree.getIOOperationCallback(), lsmBTree.getDiskComponents());
+                accessor.scheduleMerge(lsmBTree.getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index a8e0aee..2462c85 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -36,7 +36,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
@@ -48,10 +50,14 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
 import org.apache.hyracks.test.support.TestUtils;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.trace.ITracer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class LSMTreeRunner implements IExperimentRunner {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final int MAX_OPEN_FILES = Integer.MAX_VALUE;
     private static final int HYRACKS_FRAME_SIZE = 131072;
 
@@ -103,8 +109,17 @@
             virtualBufferCaches.add(virtualBufferCache);
         }
 
-        this.ioScheduler = AsynchronousScheduler.INSTANCE;
-        AsynchronousScheduler.INSTANCE.init(threadFactory);
+        this.ioScheduler = new AsynchronousScheduler(threadFactory, new IIoOperationFailedCallback() {
+            @Override
+            public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                LOGGER.error("Operation {} failed", operation, t);
+            }
+
+            @Override
+            public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                ExitUtil.exit(ExitUtil.EC_IO_SCHEDULER_FAILED);
+            }
+        });
 
         lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 803c5cb..707628b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -36,14 +36,14 @@
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -95,12 +95,12 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
         this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
-        this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+        this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
     }
 
     public void setUp() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 2fd1dbc..54fd3bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -197,7 +196,7 @@
         Mockito.doAnswer(new Answer<Void>() {
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 if (mergedSizes != null) {
                     mergedComponents.forEach(component -> {
                         mergedSizes.add(component.getComponentSize());
@@ -219,8 +218,7 @@
                 }
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 4657109..1a882b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -56,8 +56,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index 584de0c..f5a7c9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -58,8 +58,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
index a420ba9..da87b27 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
@@ -84,6 +85,12 @@
             LSMInvertedIndexTestUtils.compareActualAndExpectedIndexes(testCtx);
         }
         LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesRangeSearch(testCtx);
+        if (invIndexType == InvertedIndexType.LSM || invIndexType == InvertedIndexType.PARTITIONED_LSM) {
+            LSMInvertedIndex lsmIndex = (LSMInvertedIndex) invIndex;
+            if (!lsmIndex.isMemoryComponentsAllocated() || lsmIndex.isCurrentMutableComponentEmpty()) {
+                LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesMergeSearch(testCtx);
+            }
+        }
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 8e41c86..c1e1b4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -43,7 +43,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -92,7 +92,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 78b1658..6b06f4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -138,7 +138,7 @@
             }
 
             case MERGE: {
-                accessor.scheduleMerge(invIndex.getIOOperationCallback(), invIndex.getDiskComponents());
+                accessor.scheduleMerge(invIndex.getDiskComponents());
                 break;
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index ae8713c..2435cc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -55,6 +55,9 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils.HyracksTaskTestContext;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.util.trace.ITraceCategoryRegistry;
+import org.apache.hyracks.util.trace.TraceCategoryRegistry;
+import org.apache.hyracks.util.trace.Tracer;
 
 @SuppressWarnings("rawtypes")
 public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
@@ -184,7 +187,9 @@
                         harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
                         filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
                 break;
             }
             case PARTITIONED_LSM: {
@@ -195,7 +200,9 @@
                         harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
                         filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
                 break;
             }
             default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index db3bb50..2f58259 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -19,7 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.util;
 
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
@@ -87,6 +87,8 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexMergeCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
@@ -276,17 +278,33 @@
         }
     }
 
+    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        IInvertedIndexAccessor invIndexAccessor =
+                (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        compareActualAndExpectedIndexesRangeSearch(testCtx, invIndexAccessor.createRangeSearchCursor());
+    }
+
+    public static void compareActualAndExpectedIndexesMergeSearch(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        LSMInvertedIndexAccessor invIndexAccessor =
+                (LSMInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        compareActualAndExpectedIndexesRangeSearch(testCtx,
+                new LSMInvertedIndexMergeCursor(invIndexAccessor.getOpContext()));
+    }
+
     /**
      * Compares actual and expected indexes using the rangeSearch() method of the inverted-index accessor.
      */
-    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
-            throws HyracksDataException {
+    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx,
+            IIndexCursor invIndexCursor) throws HyracksDataException {
         IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
         int tokenFieldCount = invIndex.getTokenTypeTraits().length;
         int invListFieldCount = invIndex.getInvListTypeTraits().length;
         IInvertedIndexAccessor invIndexAccessor =
                 (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        IIndexCursor invIndexCursor = invIndexAccessor.createRangeSearchCursor();
         try {
             MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
             IBinaryComparatorFactory[] tupleCmpFactories =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 0e0f7e5..ee044bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -76,8 +76,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
 
             rTreeTestUtils.checkScan(ctx);
             rTreeTestUtils.checkDiskOrderScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 581dbf7..1fa9d95 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -25,7 +25,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
@@ -75,9 +74,8 @@
     }
 
     protected void flush(AbstractRTreeTestContext ctx) throws HyracksDataException {
-        ILSMIndex lsmIndex = (ILSMIndex) ctx.getIndex();
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+        accessor.scheduleFlush();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 2a14780..810c3c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -78,7 +78,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(lsmRTree.getIOOperationCallback(), lsmRTree.getDiskComponents());
+                accessor.scheduleMerge(lsmRTree.getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index 653677c..595d675 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -66,8 +66,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getIOOperationCallback(),
-                        ((AbstractLSMRTree) lsmRTree).getDiskComponents());
+                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 8a5d0c5..598cf18 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -42,7 +42,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -89,7 +89,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index 212673e..ff5f622 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -79,10 +79,22 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-core</artifactId>
     </dependency>
-      <dependency>
-          <groupId>commons-io</groupId>
-          <artifactId>commons-io</artifactId>
-      </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-text</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java
new file mode 100644
index 0000000..147e871
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+public interface ComputingAction<T> {
+    @SuppressWarnings("squid:S00112")
+    T compute() throws Throwable;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java
new file mode 100644
index 0000000..d2023e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+public class CountRetryPolicy implements IRetryPolicy {
+
+    private final int count;
+    private int attempted = 0;
+
+    public CountRetryPolicy(int count) {
+        this.count = count;
+    }
+
+    @Override
+    public boolean retry(Throwable failure) {
+        if (attempted < count) {
+            attempted++;
+            return true;
+        }
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 1a17012..8500842 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -33,16 +33,16 @@
     public static final int EC_ABNORMAL_TERMINATION = 1;
     public static final int EC_FAILED_TO_STARTUP = 2;
     public static final int EC_FAILED_TO_RECOVER = 3;
-    public static final int NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS = 4;
+    public static final int EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS = 4;
+    public static final int EC_FAILED_TO_PROCESS_UN_INTERRUPTIBLE_REQUEST = 5;
     public static final int EC_UNHANDLED_EXCEPTION = 11;
     public static final int EC_IMMEDIATE_HALT = 33;
     public static final int EC_HALT_ABNORMAL_RESERVED_44 = 44;
-    public static final int EC_HALT_ABNORMAL_RESERVED_55 = 55;
+    public static final int EC_IO_SCHEDULER_FAILED = 55;
     public static final int EC_HALT_SHUTDOWN_TIMED_OUT = 66;
     public static final int EC_HALT_WATCHDOG_FAILED = 77;
-    public static final int EC_HALT_ABNORMAL_RESERVED_88 = 88;
+    public static final int EC_FLUSH_FAILED = 88;
     public static final int EC_TERMINATE_NC_SERVICE_DIRECTIVE = 99;
-
     private static final ExitThread exitThread = new ExitThread();
     private static final ShutdownWatchdog watchdogThread = new ShutdownWatchdog();
     private static final MutableLong shutdownHaltDelay = new MutableLong(10 * 60 * 1000L); // 10 minutes default
@@ -75,8 +75,8 @@
         exit(status);
     }
 
-    public static void halt(int status) {
-        LOGGER.fatal("JVM halting with status " + status + "; bye!", new Throwable("halt stacktrace"));
+    public static synchronized void halt(int status) {
+        LOGGER.fatal("JVM halting with status {}; thread dump at halt: {}", status, ThreadDumpUtil.takeDumpString());
         // try to give time for the log to be emitted...
         LogManager.shutdown();
         Runtime.getRuntime().halt(status);
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java
new file mode 100644
index 0000000..b266f01
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+@FunctionalInterface
+public interface IDelay {
+
+    long calculate(long attempt);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
new file mode 100644
index 0000000..29469d5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+@FunctionalInterface
+public interface IRetryPolicy {
+    /**
+     * @param failure
+     *            the cause of the failure
+     * @return true if one more attempt should be done
+     */
+    boolean retry(Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
index 6085c1c..baa3174 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
@@ -149,7 +149,7 @@
         return quoteAndEscape(new StringBuilder(), str).toString();
     }
 
-    private static StringBuilder quoteAndEscape(StringBuilder sb, String str) {
+    public static StringBuilder quoteAndEscape(StringBuilder sb, String str) {
         return escape(sb.append('"'), str).append('"');
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
index 4c6ff57..247c001 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
@@ -106,4 +106,13 @@
         }
     }
 
+    public static String getBootClassPath() {
+        try {
+            return runtimeMXBean.getBootClassPath();
+        } catch (UnsupportedOperationException e) {
+            // boot classpath is not supported in Java 9 and later
+            LOGGER.debug("ignoring exception calling RuntimeMXBean.getBootClassPath; returning null", e);
+            return null;
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
index f9f45c1..1863b28 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
@@ -18,10 +18,19 @@
  */
 package org.apache.hyracks.util;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.StandardSocketOptions;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.http.HttpHost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.conn.util.InetAddressUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -37,13 +46,65 @@
         sc.setOption(StandardSocketOptions.SO_KEEPALIVE, true);
     }
 
-    public static void closeQuietly(SocketChannel sc) {
-        if (sc.isOpen()) {
+    public static void closeQuietly(Closeable closeable) {
+        if (closeable != null) {
             try {
-                sc.close();
+                closeable.close();
             } catch (IOException e) {
-                LOGGER.warn("Failed to close socket", e);
+                LOGGER.warn("Failed to close", e);
             }
         }
     }
+
+    public static URI toUri(HttpHost host) throws URISyntaxException {
+        return builderFrom(host).build();
+    }
+
+    public static URI toUri(HttpHost host, String path) throws URISyntaxException {
+        return builderFrom(host).setPath(path).build();
+    }
+
+    public static URIBuilder builderFrom(HttpHost host) {
+        return new URIBuilder().setHost(host.getHostName()).setPort(host.getPort()).setScheme(host.getSchemeName());
+    }
+
+    public static String toHostPort(String host, String port) {
+        return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+    }
+
+    public static String toHostPort(String host, int port) {
+        return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+    }
+
+    public static String toHostPort(InetSocketAddress address) {
+        return toHostPort(address.getHostString(), address.getPort());
+    }
+
+    public static InetSocketAddress parseInetSocketAddress(String hostPortString) {
+        int lastColon = hostPortString.lastIndexOf(':');
+        String host = decodeIPv6LiteralHost(lastColon < 0 ? hostPortString : hostPortString.substring(0, lastColon));
+        int port = lastColon < 0 ? 0 : Integer.parseInt(hostPortString.substring(lastColon + 1));
+        return InetSocketAddress.createUnresolved(host, port);
+    }
+
+    public static InetSocketAddress toInetSocketAddress(String maybeLiteralHost, int port) {
+        return InetSocketAddress.createUnresolved(decodeIPv6LiteralHost(maybeLiteralHost), port);
+    }
+
+    public static List<InetSocketAddress> parseInetSocketAddresses(String... hostPortStrings) {
+        List<InetSocketAddress> hosts = new ArrayList<>();
+        for (String node : hostPortStrings) {
+            hosts.add(parseInetSocketAddress(node));
+        }
+        return hosts;
+    }
+
+    public static String encodeIPv6LiteralHost(String hostname) {
+        return InetAddressUtils.isIPv6Address(hostname) ? "[" + hostname + "]" : hostname;
+    }
+
+    public static String decodeIPv6LiteralHost(String hostname) {
+        return hostname.length() > 0 && hostname.charAt(0) == '[' ? hostname.substring(1, hostname.length() - 1)
+                : hostname;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java
new file mode 100644
index 0000000..3040fb1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+public class NoRetryPolicy implements IRetryPolicy {
+    public static final NoRetryPolicy INSTANCE = new NoRetryPolicy();
+
+    private NoRetryPolicy() {
+    }
+
+    @Override
+    public boolean retry(Throwable failure) {
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
index d8d6bb1..e75a961 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
@@ -41,14 +41,27 @@
         return unit.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS);
     }
 
+    /**
+     * Sleep for the minimum of the duration or the remaining of this span
+     *
+     * @param sleep
+     *            the amount to sleep
+     * @param unit
+     *            the unit of the amount
+     * @throws InterruptedException
+     */
     public void sleep(long sleep, TimeUnit unit) throws InterruptedException {
-        TimeUnit.NANOSECONDS.sleep(Math.min(elapsed(TimeUnit.NANOSECONDS), unit.toNanos(sleep)));
+        TimeUnit.NANOSECONDS.sleep(Math.min(remaining(TimeUnit.NANOSECONDS), unit.toNanos(sleep)));
     }
 
     public long remaining(TimeUnit unit) {
         return unit.convert(Long.max(spanNanos - elapsed(TimeUnit.NANOSECONDS), 0L), TimeUnit.NANOSECONDS);
     }
 
+    public void wait(Object monitor) throws InterruptedException {
+        TimeUnit.NANOSECONDS.timedWait(monitor, remaining(TimeUnit.NANOSECONDS));
+    }
+
     public void loopUntilExhausted(ThrowingAction action) throws Exception {
         loopUntilExhausted(action, 0, TimeUnit.NANOSECONDS);
     }
@@ -59,7 +72,41 @@
             if (elapsed(delayUnit) < delay) {
                 break;
             }
-            delayUnit.sleep(delay);
+            sleep(delay, delayUnit);
         }
     }
+
+    @Override
+    public String toString() {
+        long nanos = spanNanos % 1000;
+        long micros = TimeUnit.MICROSECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 1000;
+        long millis = TimeUnit.MILLISECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 1000;
+        long seconds = TimeUnit.SECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 60;
+        long minutes = TimeUnit.MINUTES.convert(spanNanos, TimeUnit.NANOSECONDS) % 60;
+        long hours = TimeUnit.HOURS.convert(spanNanos, TimeUnit.NANOSECONDS) % 24;
+        long days = TimeUnit.DAYS.convert(spanNanos, TimeUnit.NANOSECONDS);
+        StringBuilder builder = new StringBuilder();
+        if (days > 0) {
+            builder.append(days).append("d");
+        }
+        if (hours > 0) {
+            builder.append(hours).append("hr");
+        }
+        if (minutes > 0) {
+            builder.append(minutes).append("m");
+        }
+        if (seconds > 0) {
+            builder.append(seconds).append("s");
+        }
+        if (millis > 0) {
+            builder.append(millis).append("ms");
+        }
+        if (micros > 0) {
+            builder.append(micros).append("us");
+        }
+        if (nanos > 0 || builder.length() == 0) {
+            builder.append(nanos).append("ns");
+        }
+        return builder.toString();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
index b457c79..11be0ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
@@ -23,7 +23,7 @@
 import java.util.regex.Pattern;
 
 import org.apache.commons.collections4.map.LRUMap;
-import org.apache.commons.lang3.text.WordUtils;
+import org.apache.commons.text.WordUtils;
 
 public class StringUtil {
     private static final Map<String, String> CAMEL_CACHE = Collections.synchronizedMap(new LRUMap<>(1024));
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
new file mode 100644
index 0000000..e092666
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * The method on which this annotation is applied has a very high call frequency during normal execution.
+ * Careful consideration should be taken with regard to the number of objects created and any expensive operations in
+ * there.
+ */
+@Documented
+@Target({ ElementType.METHOD })
+@Retention(RetentionPolicy.SOURCE)
+public @interface CriticalPath {
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index 11fb6c0..cdb82bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -84,6 +84,11 @@
         throw new IllegalStateException();
     }
 
+    public static boolean isCharStart(byte[] b, int s) {
+        int c = b[s] & 0xff;
+        return (c >> 6) != 2;
+    }
+
     public static int getModifiedUTF8Len(char c) {
         if (c >= 0x0001 && c <= 0x007F) {
             return 1;
@@ -595,7 +600,7 @@
         out.write(tempBytes, 0, count);
     }
 
-    static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
+    public static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
             throws IOException {
         int utflen = 0;
         int count = 0;
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 62407a4..69739ba 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -78,7 +78,7 @@
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty-all</artifactId>
-        <version>4.1.6.Final</version>
+        <version>4.1.25.Final</version>
       </dependency>
       <dependency>
         <groupId>junit</groupId>
@@ -99,6 +99,20 @@
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-common</artifactId>
         <version>${hadoop.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>javax.servlet.jsp</groupId>
+            <artifactId>jsp-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>javax.servlet</groupId>
+            <artifactId>servlet-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>jdk.tools</groupId>
+            <artifactId>jdk.tools</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
@@ -128,6 +142,11 @@
         <version>2.6</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-text</artifactId>
+        <version>1.2</version>
+      </dependency>
+      <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-databind</artifactId>
         <version>2.8.4</version>
@@ -272,7 +291,7 @@
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
           <argLine>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8 -Xdebug
-            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${coverageArgLine} 
+            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${coverageArgLine}
           </argLine>
           <systemPropertyVariables>
             <log4j.configurationFile>${testLog4jConfigFile}</log4j.configurationFile>
@@ -500,12 +519,12 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.20.1</version>
+          <version>2.21.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-failsafe-plugin</artifactId>
-          <version>2.20.1</version>
+          <version>2.21.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.hyracks</groupId>
@@ -672,15 +691,6 @@
       </properties>
     </profile>
     <profile>
-      <id>java8</id>
-      <activation>
-        <jdk>1.8</jdk>
-      </activation>
-      <properties>
-        <jdk.version>1.8</jdk.version>
-      </properties>
-    </profile>
-    <profile>
       <id>integration-tests-only</id>
       <properties>
         <test.includes/>